You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by aw...@apache.org on 2015/02/13 03:20:20 UTC

[03/11] hadoop git commit: HDFS-7668. Convert site documentation from apt to markdown (Masatake Iwasaki via aw)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f1e5dc6/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDesign.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDesign.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDesign.md
new file mode 100644
index 0000000..87a9fcd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsDesign.md
@@ -0,0 +1,240 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+HDFS Architecture
+=================
+
+* [HDFS Architecture](#HDFS_Architecture)
+    * [Introduction](#Introduction)
+    * [Assumptions and Goals](#Assumptions_and_Goals)
+        * [Hardware Failure](#Hardware_Failure)
+        * [Streaming Data Access](#Streaming_Data_Access)
+        * [Large Data Sets](#Large_Data_Sets)
+        * [Simple Coherency Model](#Simple_Coherency_Model)
+        * ["Moving Computation is Cheaper than Moving Data"](#aMoving_Computation_is_Cheaper_than_Moving_Data)
+        * [Portability Across Heterogeneous Hardware and Software Platforms](#Portability_Across_Heterogeneous_Hardware_and_Software_Platforms)
+    * [NameNode and DataNodes](#NameNode_and_DataNodes)
+    * [The File System Namespace](#The_File_System_Namespace)
+    * [Data Replication](#Data_Replication)
+        * [Replica Placement: The First Baby Steps](#Replica_Placement:_The_First_Baby_Steps)
+        * [Replica Selection](#Replica_Selection)
+        * [Safemode](#Safemode)
+    * [The Persistence of File System Metadata](#The_Persistence_of_File_System_Metadata)
+    * [The Communication Protocols](#The_Communication_Protocols)
+    * [Robustness](#Robustness)
+        * [Data Disk Failure, Heartbeats and Re-Replication](#Data_Disk_Failure_Heartbeats_and_Re-Replication)
+        * [Cluster Rebalancing](#Cluster_Rebalancing)
+        * [Data Integrity](#Data_Integrity)
+        * [Metadata Disk Failure](#Metadata_Disk_Failure)
+        * [Snapshots](#Snapshots)
+    * [Data Organization](#Data_Organization)
+        * [Data Blocks](#Data_Blocks)
+        * [Staging](#Staging)
+        * [Replication Pipelining](#Replication_Pipelining)
+    * [Accessibility](#Accessibility)
+        * [FS Shell](#FS_Shell)
+        * [DFSAdmin](#DFSAdmin)
+        * [Browser Interface](#Browser_Interface)
+    * [Space Reclamation](#Space_Reclamation)
+        * [File Deletes and Undeletes](#File_Deletes_and_Undeletes)
+        * [Decrease Replication Factor](#Decrease_Replication_Factor)
+    * [References](#References)
+
+Introduction
+------------
+
+The Hadoop Distributed File System (HDFS) is a distributed file system designed to run on commodity hardware. It has many similarities with existing distributed file systems. However, the differences from other distributed file systems are significant. HDFS is highly fault-tolerant and is designed to be deployed on low-cost hardware. HDFS provides high throughput access to application data and is suitable for applications that have large data sets. HDFS relaxes a few POSIX requirements to enable streaming access to file system data. HDFS was originally built as infrastructure for the Apache Nutch web search engine project. HDFS is part of the Apache Hadoop Core project. The project URL is <http://hadoop.apache.org/>.
+
+Assumptions and Goals
+---------------------
+
+### Hardware Failure
+
+Hardware failure is the norm rather than the exception. An HDFS instance may consist of hundreds or thousands of server machines, each storing part of the file system’s data. The fact that there are a huge number of components and that each component has a non-trivial probability of failure means that some component of HDFS is always non-functional. Therefore, detection of faults and quick, automatic recovery from them is a core architectural goal of HDFS.
+
+### Streaming Data Access
+
+Applications that run on HDFS need streaming access to their data sets. They are not general purpose applications that typically run on general purpose file systems. HDFS is designed more for batch processing rather than interactive use by users. The emphasis is on high throughput of data access rather than low latency of data access. POSIX imposes many hard requirements that are not needed for applications that are targeted for HDFS. POSIX semantics in a few key areas has been traded to increase data throughput rates.
+
+### Large Data Sets
+
+Applications that run on HDFS have large data sets. A typical file in HDFS is gigabytes to terabytes in size. Thus, HDFS is tuned to support large files. It should provide high aggregate data bandwidth and scale to hundreds of nodes in a single cluster. It should support tens of millions of files in a single instance.
+
+### Simple Coherency Model
+
+HDFS applications need a write-once-read-many access model for files. A file once created, written, and closed need not be changed. This assumption simplifies data coherency issues and enables high throughput data access. A Map/Reduce application or a web crawler application fits perfectly with this model. There is a plan to support appending-writes to files in the future.
+
+### "Moving Computation is Cheaper than Moving Data"
+
+A computation requested by an application is much more efficient if it is executed near the data it operates on. This is especially true when the size of the data set is huge. This minimizes network congestion and increases the overall throughput of the system. The assumption is that it is often better to migrate the computation closer to where the data is located rather than moving the data to where the application is running. HDFS provides interfaces for applications to move themselves closer to where the data is located.
+
+### Portability Across Heterogeneous Hardware and Software Platforms
+
+HDFS has been designed to be easily portable from one platform to another. This facilitates widespread adoption of HDFS as a platform of choice for a large set of applications.
+
+NameNode and DataNodes
+----------------------
+
+HDFS has a master/slave architecture. An HDFS cluster consists of a single NameNode, a master server that manages the file system namespace and regulates access to files by clients. In addition, there are a number of DataNodes, usually one per node in the cluster, which manage storage attached to the nodes that they run on. HDFS exposes a file system namespace and allows user data to be stored in files. Internally, a file is split into one or more blocks and these blocks are stored in a set of DataNodes. The NameNode executes file system namespace operations like opening, closing, and renaming files and directories. It also determines the mapping of blocks to DataNodes. The DataNodes are responsible for serving read and write requests from the file system’s clients. The DataNodes also perform block creation, deletion, and replication upon instruction from the NameNode.
+
+![HDFS Architecture](images/hdfsarchitecture.png)
+
+The NameNode and DataNode are pieces of software designed to run on commodity machines. These machines typically run a GNU/Linux operating system (OS). HDFS is built using the Java language; any machine that supports Java can run the NameNode or the DataNode software. Usage of the highly portable Java language means that HDFS can be deployed on a wide range of machines. A typical deployment has a dedicated machine that runs only the NameNode software. Each of the other machines in the cluster runs one instance of the DataNode software. The architecture does not preclude running multiple DataNodes on the same machine but in a real deployment that is rarely the case.
+
+The existence of a single NameNode in a cluster greatly simplifies the architecture of the system. The NameNode is the arbitrator and repository for all HDFS metadata. The system is designed in such a way that user data never flows through the NameNode.
+
+The File System Namespace
+-------------------------
+
+HDFS supports a traditional hierarchical file organization. A user or an application can create directories and store files inside these directories. The file system namespace hierarchy is similar to most other existing file systems; one can create and remove files, move a file from one directory to another, or rename a file. HDFS does not yet implement user quotas or access permissions. HDFS does not support hard links or soft links. However, the HDFS architecture does not preclude implementing these features.
+
+The NameNode maintains the file system namespace. Any change to the file system namespace or its properties is recorded by the NameNode. An application can specify the number of replicas of a file that should be maintained by HDFS. The number of copies of a file is called the replication factor of that file. This information is stored by the NameNode.
+
+Data Replication
+----------------
+
+HDFS is designed to reliably store very large files across machines in a large cluster. It stores each file as a sequence of blocks; all blocks in a file except the last block are the same size. The blocks of a file are replicated for fault tolerance. The block size and replication factor are configurable per file. An application can specify the number of replicas of a file. The replication factor can be specified at file creation time and can be changed later. Files in HDFS are write-once and have strictly one writer at any time.
+
+The NameNode makes all decisions regarding replication of blocks. It periodically receives a Heartbeat and a Blockreport from each of the DataNodes in the cluster. Receipt of a Heartbeat implies that the DataNode is functioning properly. A Blockreport contains a list of all blocks on a DataNode.
+
+![HDFS DataNodes](images/hdfsdatanodes.png)
+
+### Replica Placement: The First Baby Steps
+
+The placement of replicas is critical to HDFS reliability and performance. Optimizing replica placement distinguishes HDFS from most other distributed file systems. This is a feature that needs lots of tuning and experience. The purpose of a rack-aware replica placement policy is to improve data reliability, availability, and network bandwidth utilization. The current implementation for the replica placement policy is a first effort in this direction. The short-term goals of implementing this policy are to validate it on production systems, learn more about its behavior, and build a foundation to test and research more sophisticated policies.
+
+Large HDFS instances run on a cluster of computers that commonly spread across many racks. Communication between two nodes in different racks has to go through switches. In most cases, network bandwidth between machines in the same rack is greater than network bandwidth between machines in different racks.
+
+The NameNode determines the rack id each DataNode belongs to via the process outlined in [Hadoop Rack Awareness](../hadoop-common/ClusterSetup.html#HadoopRackAwareness). A simple but non-optimal policy is to place replicas on unique racks. This prevents losing data when an entire rack fails and allows use of bandwidth from multiple racks when reading data. This policy evenly distributes replicas in the cluster which makes it easy to balance load on component failure. However, this policy increases the cost of writes because a write needs to transfer blocks to multiple racks.
+
+For the common case, when the replication factor is three, HDFS’s placement policy is to put one replica on one node in the local rack, another on a different node in the local rack, and the last on a different node in a different rack. This policy cuts the inter-rack write traffic which generally improves write performance. The chance of rack failure is far less than that of node failure; this policy does not impact data reliability and availability guarantees. However, it does reduce the aggregate network bandwidth used when reading data since a block is placed in only two unique racks rather than three. With this policy, the replicas of a file do not evenly distribute across the racks. One third of replicas are on one node, two thirds of replicas are on one rack, and the other third are evenly distributed across the remaining racks. This policy improves write performance without compromising data reliability or read performance.
+
+The current, default replica placement policy described here is a work in progress.
+
+### Replica Selection
+
+To minimize global bandwidth consumption and read latency, HDFS tries to satisfy a read request from a replica that is closest to the reader. If there exists a replica on the same rack as the reader node, then that replica is preferred to satisfy the read request. If angg/ HDFS cluster spans multiple data centers, then a replica that is resident in the local data center is preferred over any remote replica.
+
+### Safemode
+
+On startup, the NameNode enters a special state called Safemode. Replication of data blocks does not occur when the NameNode is in the Safemode state. The NameNode receives Heartbeat and Blockreport messages from the DataNodes. A Blockreport contains the list of data blocks that a DataNode is hosting. Each block has a specified minimum number of replicas. A block is considered safely replicated when the minimum number of replicas of that data block has checked in with the NameNode. After a configurable percentage of safely replicated data blocks checks in with the NameNode (plus an additional 30 seconds), the NameNode exits the Safemode state. It then determines the list of data blocks (if any) that still have fewer than the specified number of replicas. The NameNode then replicates these blocks to other DataNodes.
+
+The Persistence of File System Metadata
+---------------------------------------
+
+The HDFS namespace is stored by the NameNode. The NameNode uses a transaction log called the EditLog to persistently record every change that occurs to file system metadata. For example, creating a new file in HDFS causes the NameNode to insert a record into the EditLog indicating this. Similarly, changing the replication factor of a file causes a new record to be inserted into the EditLog. The NameNode uses a file in its local host OS file system to store the EditLog. The entire file system namespace, including the mapping of blocks to files and file system properties, is stored in a file called the FsImage. The FsImage is stored as a file in the NameNode’s local file system too.
+
+The NameNode keeps an image of the entire file system namespace and file Blockmap in memory. This key metadata item is designed to be compact, such that a NameNode with 4 GB of RAM is plenty to support a huge number of files and directories. When the NameNode starts up, it reads the FsImage and EditLog from disk, applies all the transactions from the EditLog to the in-memory representation of the FsImage, and flushes out this new version into a new FsImage on disk. It can then truncate the old EditLog because its transactions have been applied to the persistent FsImage. This process is called a checkpoint. In the current implementation, a checkpoint only occurs when the NameNode starts up. Work is in progress to support periodic checkpointing in the near future.
+
+The DataNode stores HDFS data in files in its local file system. The DataNode has no knowledge about HDFS files. It stores each block of HDFS data in a separate file in its local file system. The DataNode does not create all files in the same directory. Instead, it uses a heuristic to determine the optimal number of files per directory and creates subdirectories appropriately. It is not optimal to create all local files in the same directory because the local file system might not be able to efficiently support a huge number of files in a single directory. When a DataNode starts up, it scans through its local file system, generates a list of all HDFS data blocks that correspond to each of these local files and sends this report to the NameNode: this is the Blockreport.
+
+The Communication Protocols
+---------------------------
+
+All HDFS communication protocols are layered on top of the TCP/IP protocol. A client establishes a connection to a configurable TCP port on the NameNode machine. It talks the ClientProtocol with the NameNode. The DataNodes talk to the NameNode using the DataNode Protocol. A Remote Procedure Call (RPC) abstraction wraps both the Client Protocol and the DataNode Protocol. By design, the NameNode never initiates any RPCs. Instead, it only responds to RPC requests issued by DataNodes or clients.
+
+Robustness
+----------
+
+The primary objective of HDFS is to store data reliably even in the presence of failures. The three common types of failures are NameNode failures, DataNode failures and network partitions.
+
+### Data Disk Failure, Heartbeats and Re-Replication
+
+Each DataNode sends a Heartbeat message to the NameNode periodically. A network partition can cause a subset of DataNodes to lose connectivity with the NameNode. The NameNode detects this condition by the absence of a Heartbeat message. The NameNode marks DataNodes without recent Heartbeats as dead and does not forward any new IO requests to them. Any data that was registered to a dead DataNode is not available to HDFS any more. DataNode death may cause the replication factor of some blocks to fall below their specified value. The NameNode constantly tracks which blocks need to be replicated and initiates replication whenever necessary. The necessity for re-replication may arise due to many reasons: a DataNode may become unavailable, a replica may become corrupted, a hard disk on a DataNode may fail, or the replication factor of a file may be increased.
+
+### Cluster Rebalancing
+
+The HDFS architecture is compatible with data rebalancing schemes. A scheme might automatically move data from one DataNode to another if the free space on a DataNode falls below a certain threshold. In the event of a sudden high demand for a particular file, a scheme might dynamically create additional replicas and rebalance other data in the cluster. These types of data rebalancing schemes are not yet implemented.
+
+### Data Integrity
+
+It is possible that a block of data fetched from a DataNode arrives corrupted. This corruption can occur because of faults in a storage device, network faults, or buggy software. The HDFS client software implements checksum checking on the contents of HDFS files. When a client creates an HDFS file, it computes a checksum of each block of the file and stores these checksums in a separate hidden file in the same HDFS namespace. When a client retrieves file contents it verifies that the data it received from each DataNode matches the checksum stored in the associated checksum file. If not, then the client can opt to retrieve that block from another DataNode that has a replica of that block.
+
+### Metadata Disk Failure
+
+The FsImage and the EditLog are central data structures of HDFS. A corruption of these files can cause the HDFS instance to be non-functional. For this reason, the NameNode can be configured to support maintaining multiple copies of the FsImage and EditLog. Any update to either the FsImage or EditLog causes each of the FsImages and EditLogs to get updated synchronously. This synchronous updating of multiple copies of the FsImage and EditLog may degrade the rate of namespace transactions per second that a NameNode can support. However, this degradation is acceptable because even though HDFS applications are very data intensive in nature, they are not metadata intensive. When a NameNode restarts, it selects the latest consistent FsImage and EditLog to use.
+
+The NameNode machine is a single point of failure for an HDFS cluster. If the NameNode machine fails, manual intervention is necessary. Currently, automatic restart and failover of the NameNode software to another machine is not supported.
+
+### Snapshots
+
+Snapshots support storing a copy of data at a particular instant of time. One usage of the snapshot feature may be to roll back a corrupted HDFS instance to a previously known good point in time. HDFS does not currently support snapshots but will in a future release.
+
+Data Organization
+-----------------
+
+### Data Blocks
+
+HDFS is designed to support very large files. Applications that are compatible with HDFS are those that deal with large data sets. These applications write their data only once but they read it one or more times and require these reads to be satisfied at streaming speeds. HDFS supports write-once-read-many semantics on files. A typical block size used by HDFS is 64 MB. Thus, an HDFS file is chopped up into 64 MB chunks, and if possible, each chunk will reside on a different DataNode.
+
+### Staging
+
+A client request to create a file does not reach the NameNode immediately. In fact, initially the HDFS client caches the file data into a temporary local file. Application writes are transparently redirected to this temporary local file. When the local file accumulates data worth over one HDFS block size, the client contacts the NameNode. The NameNode inserts the file name into the file system hierarchy and allocates a data block for it. The NameNode responds to the client request with the identity of the DataNode and the destination data block. Then the client flushes the block of data from the local temporary file to the specified DataNode. When a file is closed, the remaining un-flushed data in the temporary local file is transferred to the DataNode. The client then tells the NameNode that the file is closed. At this point, the NameNode commits the file creation operation into a persistent store. If the NameNode dies before the file is closed, the file is lost.
+
+The above approach has been adopted after careful consideration of target applications that run on HDFS. These applications need streaming writes to files. If a client writes to a remote file directly without any client side buffering, the network speed and the congestion in the network impacts throughput considerably. This approach is not without precedent. Earlier distributed file systems, e.g. AFS, have used client side caching to improve performance. A POSIX requirement has been relaxed to achieve higher performance of data uploads.
+
+### Replication Pipelining
+
+When a client is writing data to an HDFS file, its data is first written to a local file as explained in the previous section. Suppose the HDFS file has a replication factor of three. When the local file accumulates a full block of user data, the client retrieves a list of DataNodes from the NameNode. This list contains the DataNodes that will host a replica of that block. The client then flushes the data block to the first DataNode. The first DataNode starts receiving the data in small portions, writes each portion to its local repository and transfers that portion to the second DataNode in the list. The second DataNode, in turn starts receiving each portion of the data block, writes that portion to its repository and then flushes that portion to the third DataNode. Finally, the third DataNode writes the data to its local repository. Thus, a DataNode can be receiving data from the previous one in the pipeline and at the same time forwarding data to the next one in the pipeline. Thu
 s, the data is pipelined from one DataNode to the next.
+
+Accessibility
+-------------
+
+HDFS can be accessed from applications in many different ways. Natively, HDFS provides a [FileSystem Java API](http://hadoop.apache.org/docs/current/api/) for applications to use. A C language wrapper for this Java API is also available. In addition, an HTTP browser can also be used to browse the files of an HDFS instance. Work is in progress to expose HDFS through the WebDAV protocol.
+
+### FS Shell
+
+HDFS allows user data to be organized in the form of files and directories. It provides a commandline interface called FS shell that lets a user interact with the data in HDFS. The syntax of this command set is similar to other shells (e.g. bash, csh) that users are already familiar with. Here are some sample action/command pairs:
+
+| Action | Command |
+|:---- |:---- |
+| Create a directory named `/foodir` | `bin/hadoop dfs -mkdir /foodir` |
+| Remove a directory named `/foodir` | `bin/hadoop fs -rm -R /foodir` |
+| View the contents of a file named `/foodir/myfile.txt` | `bin/hadoop dfs -cat /foodir/myfile.txt` |
+
+FS shell is targeted for applications that need a scripting language to interact with the stored data.
+
+### DFSAdmin
+
+The DFSAdmin command set is used for administering an HDFS cluster. These are commands that are used only by an HDFS administrator. Here are some sample action/command pairs:
+
+| Action | Command |
+|:---- |:---- |
+| Put the cluster in Safemode | `bin/hdfs dfsadmin -safemode enter` |
+| Generate a list of DataNodes | `bin/hdfs dfsadmin -report` |
+| Recommission or decommission DataNode(s) | `bin/hdfs dfsadmin -refreshNodes` |
+
+### Browser Interface
+
+A typical HDFS install configures a web server to expose the HDFS namespace through a configurable TCP port. This allows a user to navigate the HDFS namespace and view the contents of its files using a web browser.
+
+Space Reclamation
+-----------------
+
+### File Deletes and Undeletes
+
+When a file is deleted by a user or an application, it is not immediately removed from HDFS. Instead, HDFS first renames it to a file in the `/trash` directory. The file can be restored quickly as long as it remains in `/trash`. A file remains in `/trash` for a configurable amount of time. After the expiry of its life in `/trash`, the NameNode deletes the file from the HDFS namespace. The deletion of a file causes the blocks associated with the file to be freed. Note that there could be an appreciable time delay between the time a file is deleted by a user and the time of the corresponding increase in free space in HDFS.
+
+A user can Undelete a file after deleting it as long as it remains in the `/trash` directory. If a user wants to undelete a file that he/she has deleted, he/she can navigate the `/trash` directory and retrieve the file. The `/trash` directory contains only the latest copy of the file that was deleted. The `/trash` directory is just like any other directory with one special feature: HDFS applies specified policies to automatically delete files from this directory. Current default trash interval is set to 0 (Deletes file without storing in trash). This value is configurable parameter stored as `fs.trash.interval` stored in core-site.xml.
+
+### Decrease Replication Factor
+
+When the replication factor of a file is reduced, the NameNode selects excess replicas that can be deleted. The next Heartbeat transfers this information to the DataNode. The DataNode then removes the corresponding blocks and the corresponding free space appears in the cluster. Once again, there might be a time delay between the completion of the setReplication API call and the appearance of free space in the cluster.
+
+References
+----------
+
+Hadoop [JavaDoc API](http://hadoop.apache.org/docs/current/api/).
+
+HDFS source code: <http://hadoop.apache.org/version_control.html>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f1e5dc6/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsEditsViewer.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsEditsViewer.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsEditsViewer.md
new file mode 100644
index 0000000..267471f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsEditsViewer.md
@@ -0,0 +1,69 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+Offline Edits Viewer Guide
+==========================
+
+* [Offline Edits Viewer Guide](#Offline_Edits_Viewer_Guide)
+    * [Overview](#Overview)
+    * [Usage](#Usage)
+    * [Case study: Hadoop cluster recovery](#Case_study:_Hadoop_cluster_recovery)
+
+Overview
+--------
+
+Offline Edits Viewer is a tool to parse the Edits log file. The current processors are mostly useful for conversion between different formats, including XML which is human readable and easier to edit than native binary format.
+
+The tool can parse the edits formats -18 (roughly Hadoop 0.19) and later. The tool operates on files only, it does not need Hadoop cluster to be running.
+
+Input formats supported:
+
+1.  **binary**: native binary format that Hadoop uses internally
+2.  **xml**: XML format, as produced by xml processor, used if filename
+    has `.xml` (case insensitive) extension
+
+The Offline Edits Viewer provides several output processors (unless stated otherwise the output of the processor can be converted back to original edits file):
+
+1.  **binary**: native binary format that Hadoop uses internally
+2.  **xml**: XML format
+3.  **stats**: prints out statistics, this cannot be converted back to
+    Edits file
+
+Usage
+-----
+
+       bash$ bin/hdfs oev -i edits -o edits.xml
+
+|                                       Flag | Description |
+|:---- |:---- |
+| [`-i` ; `--inputFile`] *input file* | Specify the input edits log file to process. Xml (case insensitive) extension means XML format otherwise binary format is assumed. Required. |
+| [`-o` ; `--outputFile`] *output file* | Specify the output filename, if the specified output processor generates one. If the specified file already exists, it is silently overwritten. Required. |
+| [`-p` ; `--processor`] *processor* | Specify the image processor to apply against the image file. Currently valid options are `binary`, `xml` (default) and `stats`. |
+| [`-v` ; `--verbose`] | Print the input and output filenames and pipe output of processor to console as well as specified file. On extremely large files, this may increase processing time by an order of magnitude. |
+| [`-h` ; `--help`] | Display the tool usage and help information and exit. |
+
+Case study: Hadoop cluster recovery
+-----------------------------------
+
+In case there is some problem with hadoop cluster and the edits file is corrupted it is possible to save at least part of the edits file that is correct. This can be done by converting the binary edits to XML, edit it manually and then convert it back to binary. The most common problem is that the edits file is missing the closing record (record that has opCode -1). This should be recognized by the tool and the XML format should be properly closed.
+
+If there is no closing record in the XML file you can add one after last correct record. Anything after the record with opCode -1 is ignored.
+
+Example of a closing record (with opCode -1):
+
+      <RECORD>
+        <OPCODE>-1</OPCODE>
+        <DATA>
+        </DATA>
+      </RECORD>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f1e5dc6/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
new file mode 100644
index 0000000..864eee8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
@@ -0,0 +1,172 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+Offline Image Viewer Guide
+==========================
+
+* [Offline Image Viewer Guide](#Offline_Image_Viewer_Guide)
+    * [Overview](#Overview)
+    * [Usage](#Usage)
+        * [Web Processor](#Web_Processor)
+        * [XML Processor](#XML_Processor)
+    * [Options](#Options)
+    * [Analyzing Results](#Analyzing_Results)
+    * [oiv\_legacy Command](#oiv_legacy_Command)
+        * [Usage](#Usage)
+        * [Options](#Options)
+
+Overview
+--------
+
+The Offline Image Viewer is a tool to dump the contents of hdfs fsimage files to a human-readable format and provide read-only WebHDFS API in order to allow offline analysis and examination of an Hadoop cluster's namespace. The tool is able to process very large image files relatively quickly. The tool handles the layout formats that were included with Hadoop versions 2.4 and up. If you want to handle older layout formats, you can use the Offline Image Viewer of Hadoop 2.3 or [oiv\_legacy Command](#oiv_legacy_Command). If the tool is not able to process an image file, it will exit cleanly. The Offline Image Viewer does not require a Hadoop cluster to be running; it is entirely offline in its operation.
+
+The Offline Image Viewer provides several output processors:
+
+1.  Web is the default output processor. It launches a HTTP server
+    that exposes read-only WebHDFS API. Users can investigate the namespace
+    interactively by using HTTP REST API.
+
+2.  XML creates an XML document of the fsimage and includes all of the
+    information within the fsimage, similar to the lsr processor. The
+    output of this processor is amenable to automated processing and
+    analysis with XML tools. Due to the verbosity of the XML syntax,
+    this processor will also generate the largest amount of output.
+
+3.  FileDistribution is the tool for analyzing file sizes in the
+    namespace image. In order to run the tool one should define a range
+    of integers [0, maxSize] by specifying maxSize and a step. The
+    range of integers is divided into segments of size step: [0, s[1],
+    ..., s[n-1], maxSize], and the processor calculates how many files
+    in the system fall into each segment [s[i-1], s[i]). Note that
+    files larger than maxSize always fall into the very last segment.
+    The output file is formatted as a tab separated two column table:
+    Size and NumFiles. Where Size represents the start of the segment,
+    and numFiles is the number of files form the image which size falls
+    in this segment.
+
+Usage
+-----
+
+### Web Processor
+
+Web processor launches a HTTP server which exposes read-only WebHDFS API. Users can specify the address to listen by -addr option (default by localhost:5978).
+
+       bash$ bin/hdfs oiv -i fsimage
+       14/04/07 13:25:14 INFO offlineImageViewer.WebImageViewer: WebImageViewer
+       started. Listening on /127.0.0.1:5978. Press Ctrl+C to stop the viewer.
+
+Users can access the viewer and get the information of the fsimage by the following shell command:
+
+       bash$ bin/hdfs dfs -ls webhdfs://127.0.0.1:5978/
+       Found 2 items
+       drwxrwx--* - root supergroup          0 2014-03-26 20:16 webhdfs://127.0.0.1:5978/tmp
+       drwxr-xr-x   - root supergroup          0 2014-03-31 14:08 webhdfs://127.0.0.1:5978/user
+
+To get the information of all the files and directories, you can simply use the following command:
+
+       bash$ bin/hdfs dfs -ls -R webhdfs://127.0.0.1:5978/
+
+Users can also get JSON formatted FileStatuses via HTTP REST API.
+
+       bash$ curl -i http://127.0.0.1:5978/webhdfs/v1/?op=liststatus
+       HTTP/1.1 200 OK
+       Content-Type: application/json
+       Content-Length: 252
+
+       {"FileStatuses":{"FileStatus":[
+       {"fileId":16386,"accessTime":0,"replication":0,"owner":"theuser","length":0,"permission":"755","blockSize":0,"modificationTime":1392772497282,"type":"DIRECTORY","group":"supergroup","childrenNum":1,"pathSuffix":"user"}
+       ]}}
+
+The Web processor now supports the following operations:
+
+* [LISTSTATUS](./WebHDFS.html#List_a_Directory)
+* [GETFILESTATUS](./WebHDFS.html#Status_of_a_FileDirectory)
+* [GETACLSTATUS](./WebHDFS.html#Get_ACL_Status)
+
+### XML Processor
+
+XML Processor is used to dump all the contents in the fsimage. Users can specify input and output file via -i and -o command-line.
+
+       bash$ bin/hdfs oiv -p XML -i fsimage -o fsimage.xml
+
+This will create a file named fsimage.xml contains all the information in the fsimage. For very large image files, this process may take several minutes.
+
+Applying the Offline Image Viewer with XML processor would result in the following output:
+
+       <?xml version="1.0"?>
+       <fsimage>
+       <NameSection>
+         <genstampV1>1000</genstampV1>
+         <genstampV2>1002</genstampV2>
+         <genstampV1Limit>0</genstampV1Limit>
+         <lastAllocatedBlockId>1073741826</lastAllocatedBlockId>
+         <txid>37</txid>
+       </NameSection>
+       <INodeSection>
+         <lastInodeId>16400</lastInodeId>
+         <inode>
+           <id>16385</id>
+           <type>DIRECTORY</type>
+           <name></name>
+           <mtime>1392772497282</mtime>
+           <permission>theuser:supergroup:rwxr-xr-x</permission>
+           <nsquota>9223372036854775807</nsquota>
+           <dsquota>-1</dsquota>
+         </inode>
+       ...remaining output omitted...
+
+Options
+-------
+
+| **Flag** | **Description** |
+|:---- |:---- |
+| `-i`\|`--inputFile` *input file* | Specify the input fsimage file to process. Required. |
+| `-o`\|`--outputFile` *output file* | Specify the output filename, if the specified output processor generates one. If the specified file already exists, it is silently overwritten. (output to stdout by default)\|
+| `-p`\|`--processor` *processor* | Specify the image processor to apply against the image file. Currently valid options are Web (default), XML and FileDistribution. |
+| `-addr` *address* | Specify the address(host:port) to listen. (localhost:5978 by default). This option is used with Web processor. |
+| `-maxSize` *size* | Specify the range [0, maxSize] of file sizes to be analyzed in bytes (128GB by default). This option is used with FileDistribution processor. |
+| `-step` *size* | Specify the granularity of the distribution in bytes (2MB by default). This option is used with FileDistribution processor. |
+| `-h`\|`--help` | Display the tool usage and help information and exit. |
+
+Analyzing Results
+-----------------
+
+The Offline Image Viewer makes it easy to gather large amounts of data about the hdfs namespace. This information can then be used to explore file system usage patterns or find specific files that match arbitrary criteria, along with other types of namespace analysis.
+
+oiv\_legacy Command
+-------------------
+
+Due to the internal layout changes introduced by the ProtocolBuffer-based fsimage ([HDFS-5698](https://issues.apache.org/jira/browse/HDFS-5698)), OfflineImageViewer consumes excessive amount of memory and loses some functions such as Indented and Delimited processor. If you want to process without large amount of memory or use these processors, you can use `oiv_legacy` command (same as `oiv` in Hadoop 2.3).
+
+### Usage
+
+1. Set `dfs.namenode.legacy-oiv-image.dir` to an appropriate directory
+   to make standby NameNode or SecondaryNameNode save its namespace in the
+   old fsimage format during checkpointing.
+
+2. Use `oiv_legacy` command to the old format fsimage.
+
+        bash$ bin/hdfs oiv_legacy -i fsimage_old -o output
+
+### Options
+
+| **Flag** | **Description** |
+|:---- |:---- |
+| `-i`\|`--inputFile` *input file* | Specify the input fsimage file to process. Required. |
+| `-o`\|`--outputFile` *output file* | Specify the output filename, if the specified output processor generates one. If the specified file already exists, it is silently overwritten. Required. |
+| `-p`\|`--processor` *processor* | Specify the image processor to apply against the image file. Valid options are Ls (default), XML, Delimited, Indented, and FileDistribution. |
+| `-skipBlocks` | Do not enumerate individual blocks within files. This may save processing time and outfile file space on namespaces with very large files. The Ls processor reads the blocks to correctly determine file sizes and ignores this option. |
+| `-printToScreen` | Pipe output of processor to console as well as specified file. On extremely large namespaces, this may increase processing time by an order of magnitude. |
+| `-delimiter` *arg* | When used in conjunction with the Delimited processor, replaces the default tab delimiter with the string specified by *arg*. |
+| `-h`\|`--help` | Display the tool usage and help information and exit. |

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f1e5dc6/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsMultihoming.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsMultihoming.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsMultihoming.md
new file mode 100644
index 0000000..fc7cc3d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsMultihoming.md
@@ -0,0 +1,127 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+HDFS Support for Multihomed Networks
+====================================
+
+This document is targetted to cluster administrators deploying `HDFS` in multihomed networks. Similar support for `YARN`/`MapReduce` is work in progress and will be documented when available.
+
+* [HDFS Support for Multihomed Networks](#HDFS_Support_for_Multihomed_Networks)
+    * [Multihoming Background](#Multihoming_Background)
+    * [Fixing Hadoop Issues In Multihomed Environments](#Fixing_Hadoop_Issues_In_Multihomed_Environments)
+        * [Ensuring HDFS Daemons Bind All Interfaces](#Ensuring_HDFS_Daemons_Bind_All_Interfaces)
+        * [Clients use Hostnames when connecting to DataNodes](#Clients_use_Hostnames_when_connecting_to_DataNodes)
+        * [DataNodes use HostNames when connecting to other DataNodes](#DataNodes_use_HostNames_when_connecting_to_other_DataNodes)
+
+Multihoming Background
+----------------------
+
+In multihomed networks the cluster nodes are connected to more than one network interface. There could be multiple reasons for doing so.
+
+1.  **Security**: Security requirements may dictate that intra-cluster
+    traffic be confined to a different network than the network used to
+    transfer data in and out of the cluster.
+
+2.  **Performance**: Intra-cluster traffic may use one or more high bandwidth
+    interconnects like Fiber Channel, Infiniband or 10GbE.
+
+3.  **Failover/Redundancy**: The nodes may have multiple network adapters
+    connected to a single network to handle network adapter failure.
+
+Note that NIC Bonding (also known as NIC Teaming or Link
+Aggregation) is a related but separate topic. The following settings
+are usually not applicable to a NIC bonding configuration which handles
+multiplexing and failover transparently while presenting a single 'logical
+network' to applications.
+
+Fixing Hadoop Issues In Multihomed Environments
+-----------------------------------------------
+
+### Ensuring HDFS Daemons Bind All Interfaces
+
+By default `HDFS` endpoints are specified as either hostnames or IP addresses. In either case `HDFS` daemons will bind to a single IP address making the daemons unreachable from other networks.
+
+The solution is to have separate setting for server endpoints to force binding the wildcard IP address `INADDR_ANY` i.e. `0.0.0.0`. Do NOT supply a port number with any of these settings.
+
+    <property>
+      <name>dfs.namenode.rpc-bind-host</name>
+      <value>0.0.0.0</value>
+      <description>
+        The actual address the RPC server will bind to. If this optional address is
+        set, it overrides only the hostname portion of dfs.namenode.rpc-address.
+        It can also be specified per name node or name service for HA/Federation.
+        This is useful for making the name node listen on all interfaces by
+        setting it to 0.0.0.0.
+      </description>
+    </property>
+
+    <property>
+      <name>dfs.namenode.servicerpc-bind-host</name>
+      <value>0.0.0.0</value>
+      <description>
+        The actual address the service RPC server will bind to. If this optional address is
+        set, it overrides only the hostname portion of dfs.namenode.servicerpc-address.
+        It can also be specified per name node or name service for HA/Federation.
+        This is useful for making the name node listen on all interfaces by
+        setting it to 0.0.0.0.
+      </description>
+    </property>
+
+    <property>
+      <name>dfs.namenode.http-bind-host</name>
+      <value>0.0.0.0</value>
+      <description>
+        The actual adress the HTTP server will bind to. If this optional address
+        is set, it overrides only the hostname portion of dfs.namenode.http-address.
+        It can also be specified per name node or name service for HA/Federation.
+        This is useful for making the name node HTTP server listen on all
+        interfaces by setting it to 0.0.0.0.
+      </description>
+    </property>
+
+    <property>
+      <name>dfs.namenode.https-bind-host</name>
+      <value>0.0.0.0</value>
+      <description>
+        The actual adress the HTTPS server will bind to. If this optional address
+        is set, it overrides only the hostname portion of dfs.namenode.https-address.
+        It can also be specified per name node or name service for HA/Federation.
+        This is useful for making the name node HTTPS server listen on all
+        interfaces by setting it to 0.0.0.0.
+      </description>
+    </property>
+
+### Clients use Hostnames when connecting to DataNodes
+
+By default `HDFS` clients connect to DataNodes using the IP address provided by the NameNode. Depending on the network configuration this IP address may be unreachable by the clients. The fix is letting clients perform their own DNS resolution of the DataNode hostname. The following setting enables this behavior.
+
+    <property>
+      <name>dfs.client.use.datanode.hostname</name>
+      <value>true</value>
+      <description>Whether clients should use datanode hostnames when
+        connecting to datanodes.
+      </description>
+    </property>
+
+### DataNodes use HostNames when connecting to other DataNodes
+
+Rarely, the NameNode-resolved IP address for a DataNode may be unreachable from other DataNodes. The fix is to force DataNodes to perform their own DNS resolution for inter-DataNode connections. The following setting enables this behavior.
+
+    <property>
+      <name>dfs.datanode.use.datanode.hostname</name>
+      <value>true</value>
+      <description>Whether datanodes should use datanode hostnames when
+        connecting to other datanodes for data transfer.
+      </description>
+    </property>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f1e5dc6/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsNfsGateway.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsNfsGateway.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsNfsGateway.md
new file mode 100644
index 0000000..cea491f
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsNfsGateway.md
@@ -0,0 +1,254 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+HDFS NFS Gateway
+================
+
+* [HDFS NFS Gateway](#HDFS_NFS_Gateway)
+    * [Overview](#Overview)
+    * [Configuration](#Configuration)
+    * [Start and stop NFS gateway service](#Start_and_stop_NFS_gateway_service)
+    * [Verify validity of NFS related services](#Verify_validity_of_NFS_related_services)
+    * [Mount the export "/"](#Mount_the_export_)
+    * [Allow mounts from unprivileged clients](#Allow_mounts_from_unprivileged_clients)
+    * [User authentication and mapping](#User_authentication_and_mapping)
+
+Overview
+--------
+
+The NFS Gateway supports NFSv3 and allows HDFS to be mounted as part of the client's local file system. Currently NFS Gateway supports and enables the following usage patterns:
+
+* Users can browse the HDFS file system through their local file system
+  on NFSv3 client compatible operating systems.
+* Users can download files from the the HDFS file system on to their
+  local file system.
+* Users can upload files from their local file system directly to the
+  HDFS file system.
+* Users can stream data directly to HDFS through the mount point. File
+  append is supported but random write is not supported.
+
+The NFS gateway machine needs the same thing to run an HDFS client like Hadoop JAR files, HADOOP\_CONF directory. The NFS gateway can be on the same host as DataNode, NameNode, or any HDFS client.
+
+Configuration
+-------------
+
+The NFS-gateway uses proxy user to proxy all the users accessing the NFS mounts. In non-secure mode, the user running the gateway is the proxy user, while in secure mode the user in Kerberos keytab is the proxy user. Suppose the proxy user is 'nfsserver' and users belonging to the groups 'users-group1' and 'users-group2' use the NFS mounts, then in core-site.xml of the NameNode, the following two properities must be set and only NameNode needs restart after the configuration change (NOTE: replace the string 'nfsserver' with the proxy user name in your cluster):
+
+    <property>
+      <name>hadoop.proxyuser.nfsserver.groups</name>
+      <value>root,users-group1,users-group2</value>
+      <description>
+             The 'nfsserver' user is allowed to proxy all members of the 'users-group1' and
+             'users-group2' groups. Note that in most cases you will need to include the
+             group "root" because the user "root" (which usually belonges to "root" group) will
+             generally be the user that initially executes the mount on the NFS client system.
+             Set this to '*' to allow nfsserver user to proxy any group.
+      </description>
+    </property>
+
+    <property>
+      <name>hadoop.proxyuser.nfsserver.hosts</name>
+      <value>nfs-client-host1.com</value>
+      <description>
+             This is the host where the nfs gateway is running. Set this to '*' to allow
+             requests from any hosts to be proxied.
+      </description>
+    </property>
+
+The above are the only required configuration for the NFS gateway in non-secure mode. For Kerberized hadoop clusters, the following configurations need to be added to hdfs-site.xml for the gateway (NOTE: replace string "nfsserver" with the proxy user name and ensure the user contained in the keytab is also the same proxy user):
+
+      <property>
+        <name>nfs.keytab.file</name>
+        <value>/etc/hadoop/conf/nfsserver.keytab</value> <!-- path to the nfs gateway keytab -->
+      </property>
+
+      <property>
+        <name>nfs.kerberos.principal</name>
+        <value>nfsserver/_HOST@YOUR-REALM.COM</value>
+      </property>
+
+The rest of the NFS gateway configurations are optional for both secure and non-secure mode.
+
+The AIX NFS client has a [few known issues](https://issues.apache.org/jira/browse/HDFS-6549) that prevent it from working correctly by default with the HDFS NFS Gateway. If you want to be able to access the HDFS NFS Gateway from AIX, you should set the following configuration setting to enable work-arounds for these issues:
+
+    <property>
+      <name>nfs.aix.compatibility.mode.enabled</name>
+      <value>true</value>
+    </property>
+
+Note that regular, non-AIX clients should NOT enable AIX compatibility mode. The work-arounds implemented by AIX compatibility mode effectively disable safeguards to ensure that listing of directory contents via NFS returns consistent results, and that all data sent to the NFS server can be assured to have been committed.
+
+It's strongly recommended for the users to update a few configuration properties based on their use cases. All the following configuration properties can be added or updated in hdfs-site.xml.
+
+*   If the client mounts the export with access time update allowed, make sure the following
+    property is not disabled in the configuration file. Only NameNode needs to restart after
+    this property is changed. On some Unix systems, the user can disable access time update
+    by mounting the export with "noatime". If the export is mounted with "noatime", the user
+    doesn't need to change the following property and thus no need to restart namenode.
+
+        <property>
+          <name>dfs.namenode.accesstime.precision</name>
+          <value>3600000</value>
+          <description>The access time for HDFS file is precise upto this value.
+            The default value is 1 hour. Setting a value of 0 disables
+            access times for HDFS.
+          </description>
+        </property>
+
+*   Users are expected to update the file dump directory. NFS client often
+    reorders writes. Sequential writes can arrive at the NFS gateway at random
+    order. This directory is used to temporarily save out-of-order writes
+    before writing to HDFS. For each file, the out-of-order writes are dumped after
+    they are accumulated to exceed certain threshold (e.g., 1MB) in memory.
+    One needs to make sure the directory has enough
+    space. For example, if the application uploads 10 files with each having
+    100MB, it is recommended for this directory to have roughly 1GB space in case if a
+    worst-case write reorder happens to every file. Only NFS gateway needs to restart after
+    this property is updated.
+
+          <property>
+            <name>nfs.dump.dir</name>
+            <value>/tmp/.hdfs-nfs</value>
+          </property>
+
+*   By default, the export can be mounted by any client. To better control the access,
+    users can update the following property. The value string contains machine name and
+    access privilege, separated by whitespace
+    characters. The machine name format can be a single host, a Java regular expression, or an IPv4 address. The access
+    privilege uses rw or ro to specify read/write or read-only access of the machines to exports. If the access privilege is not provided, the default is read-only. Entries are separated by ";".
+    For example: "192.168.0.0/22 rw ; host.\*\\.example\\.com ; host1.test.org ro;". Only the NFS gateway needs to restart after
+    this property is updated.
+
+        <property>
+          <name>nfs.exports.allowed.hosts</name>
+          <value>* rw</value>
+        </property>
+
+*   JVM and log settings. You can export JVM settings (e.g., heap size and GC log) in
+    HADOOP\_NFS3\_OPTS. More NFS related settings can be found in hadoop-env.sh.
+    To get NFS debug trace, you can edit the log4j.property file
+    to add the following. Note, debug trace, especially for ONCRPC, can be very verbose.
+
+    To change logging level:
+
+            log4j.logger.org.apache.hadoop.hdfs.nfs=DEBUG
+
+    To get more details of ONCRPC requests:
+
+            log4j.logger.org.apache.hadoop.oncrpc=DEBUG
+
+Start and stop NFS gateway service
+----------------------------------
+
+Three daemons are required to provide NFS service: rpcbind (or portmap), mountd and nfsd. The NFS gateway process has both nfsd and mountd. It shares the HDFS root "/" as the only export. It is recommended to use the portmap included in NFS gateway package. Even though NFS gateway works with portmap/rpcbind provide by most Linux distributions, the package included portmap is needed on some Linux systems such as REHL6.2 due to an [rpcbind bug](https://bugzilla.redhat.com/show_bug.cgi?id=731542). More detailed discussions can be found in [HDFS-4763](https://issues.apache.org/jira/browse/HDFS-4763).
+
+1.  Stop nfsv3 and rpcbind/portmap services provided by the platform (commands can be different on various Unix platforms):
+
+        [root]> service nfs stop
+        [root]> service rpcbind stop
+
+2.  Start Hadoop's portmap (needs root privileges):
+
+        [root]> $HADOOP_PREFIX/bin/hdfs --daemon start portmap
+
+3.  Start mountd and nfsd.
+
+    No root privileges are required for this command. In non-secure mode, the NFS gateway
+    should be started by the proxy user mentioned at the beginning of this user guide.
+    While in secure mode, any user can start NFS gateway
+    as long as the user has read access to the Kerberos keytab defined in "nfs.keytab.file".
+
+        [hdfs]$ $HADOOP_PREFIX/bin/hdfs --daemon start nfs3
+
+4.  Stop NFS gateway services.
+
+        [hdfs]$ $HADOOP_PREFIX/bin/hdfs --daemon stop nfs3
+        [root]> $HADOOP_PREFIX/bin/hdfs --daemon stop portmap
+
+Optionally, you can forgo running the Hadoop-provided portmap daemon and instead use the system portmap daemon on all operating systems if you start the NFS Gateway as root. This will allow the HDFS NFS Gateway to work around the aforementioned bug and still register using the system portmap daemon. To do so, just start the NFS gateway daemon as you normally would, but make sure to do so as the "root" user, and also set the "HADOOP\_PRIVILEGED\_NFS\_USER" environment variable to an unprivileged user. In this mode the NFS Gateway will start as root to perform its initial registration with the system portmap, and then will drop privileges back to the user specified by the HADOOP\_PRIVILEGED\_NFS\_USER afterward and for the rest of the duration of the lifetime of the NFS Gateway process. Note that if you choose this route, you should skip steps 1 and 2 above.
+
+Verify validity of NFS related services
+---------------------------------------
+
+1.  Execute the following command to verify if all the services are up and running:
+
+        [root]> rpcinfo -p $nfs_server_ip
+
+    You should see output similar to the following:
+
+               program vers proto   port
+
+               100005    1   tcp   4242  mountd
+
+               100005    2   udp   4242  mountd
+
+               100005    2   tcp   4242  mountd
+
+               100000    2   tcp    111  portmapper
+
+               100000    2   udp    111  portmapper
+
+               100005    3   udp   4242  mountd
+
+               100005    1   udp   4242  mountd
+
+               100003    3   tcp   2049  nfs
+
+               100005    3   tcp   4242  mountd
+
+2.  Verify if the HDFS namespace is exported and can be mounted.
+
+        [root]> showmount -e $nfs_server_ip
+
+    You should see output similar to the following:
+
+                Exports list on $nfs_server_ip :
+
+                / (everyone)
+
+Mount the export "/"
+--------------------
+
+Currently NFS v3 only uses TCP as the transportation protocol. NLM is not supported so mount option "nolock" is needed. It's recommended to use hard mount. This is because, even after the client sends all data to NFS gateway, it may take NFS gateway some extra time to transfer data to HDFS when writes were reorderd by NFS client Kernel.
+
+If soft mount has to be used, the user should give it a relatively long timeout (at least no less than the default timeout on the host) .
+
+The users can mount the HDFS namespace as shown below:
+
+     [root]>mount -t nfs -o vers=3,proto=tcp,nolock,noacl $server:/  $mount_point
+
+Then the users can access HDFS as part of the local file system except that, hard link and random write are not supported yet. To optimize the performance of large file I/O, one can increase the NFS transfer size(rsize and wsize) during mount. By default, NFS gateway supports 1MB as the maximum transfer size. For larger data transfer size, one needs to update "nfs.rtmax" and "nfs.rtmax" in hdfs-site.xml.
+
+Allow mounts from unprivileged clients
+--------------------------------------
+
+In environments where root access on client machines is not generally available, some measure of security can be obtained by ensuring that only NFS clients originating from privileged ports can connect to the NFS server. This feature is referred to as "port monitoring." This feature is not enabled by default in the HDFS NFS Gateway, but can be optionally enabled by setting the following config in hdfs-site.xml on the NFS Gateway machine:
+
+    <property>
+      <name>nfs.port.monitoring.disabled</name>
+      <value>false</value>
+    </property>
+
+User authentication and mapping
+-------------------------------
+
+NFS gateway in this release uses AUTH\_UNIX style authentication. When the user on NFS client accesses the mount point, NFS client passes the UID to NFS gateway. NFS gateway does a lookup to find user name from the UID, and then passes the username to the HDFS along with the HDFS requests. For example, if the NFS client has current user as "admin", when the user accesses the mounted directory, NFS gateway will access HDFS as user "admin". To access HDFS as the user "hdfs", one needs to switch the current user to "hdfs" on the client system when accessing the mounted directory.
+
+The system administrator must ensure that the user on NFS client host has the same name and UID as that on the NFS gateway host. This is usually not a problem if the same user management system (e.g., LDAP/NIS) is used to create and deploy users on HDFS nodes and NFS client node. In case the user account is created manually on different hosts, one might need to modify UID (e.g., do "usermod -u 123 myusername") on either NFS client or NFS gateway host in order to make it the same on both sides. More technical details of RPC AUTH\_UNIX can be found in [RPC specification](http://tools.ietf.org/html/rfc1057).
+
+Optionally, the system administrator can configure a custom static mapping file in the event one wishes to access the HDFS NFS Gateway from a system with a completely disparate set of UIDs/GIDs. By default this file is located at "/etc/nfs.map", but a custom location can be configured by setting the "static.id.mapping.file" property to the path of the static mapping file. The format of the static mapping file is similar to what is described in the exports(5) manual page, but roughly it is:
+
+    # Mapping for clients accessing the NFS gateway
+    uid 10 100 # Map the remote UID 10 the local UID 100
+    gid 11 101 # Map the remote GID 11 to the local GID 101

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f1e5dc6/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsPermissionsGuide.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsPermissionsGuide.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsPermissionsGuide.md
new file mode 100644
index 0000000..450255e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsPermissionsGuide.md
@@ -0,0 +1,284 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+HDFS Permissions Guide
+======================
+
+* [HDFS Permissions Guide](#HDFS_Permissions_Guide)
+    * [Overview](#Overview)
+    * [User Identity](#User_Identity)
+    * [Group Mapping](#Group_Mapping)
+    * [Understanding the Implementation](#Understanding_the_Implementation)
+    * [Changes to the File System API](#Changes_to_the_File_System_API)
+    * [Changes to the Application Shell](#Changes_to_the_Application_Shell)
+    * [The Super-User](#The_Super-User)
+    * [The Web Server](#The_Web_Server)
+    * [ACLs (Access Control Lists)](#ACLs_Access_Control_Lists)
+    * [ACLs File System API](#ACLs_File_System_API)
+    * [ACLs Shell Commands](#ACLs_Shell_Commands)
+    * [Configuration Parameters](#Configuration_Parameters)
+
+Overview
+--------
+
+The Hadoop Distributed File System (HDFS) implements a permissions model for files and directories that shares much of the POSIX model. Each file and directory is associated with an owner and a group. The file or directory has separate permissions for the user that is the owner, for other users that are members of the group, and for all other users. For files, the r permission is required to read the file, and the w permission is required to write or append to the file. For directories, the r permission is required to list the contents of the directory, the w permission is required to create or delete files or directories, and the x permission is required to access a child of the directory.
+
+In contrast to the POSIX model, there are no setuid or setgid bits for files as there is no notion of executable files. For directories, there are no setuid or setgid bits directory as a simplification. The Sticky bit can be set on directories, preventing anyone except the superuser, directory owner or file owner from deleting or moving the files within the directory. Setting the sticky bit for a file has no effect. Collectively, the permissions of a file or directory are its mode. In general, Unix customs for representing and displaying modes will be used, including the use of octal numbers in this description. When a file or directory is created, its owner is the user identity of the client process, and its group is the group of the parent directory (the BSD rule).
+
+HDFS also provides optional support for POSIX ACLs (Access Control Lists) to augment file permissions with finer-grained rules for specific named users or named groups. ACLs are discussed in greater detail later in this document.
+
+Each client process that accesses HDFS has a two-part identity composed of the user name, and groups list. Whenever HDFS must do a permissions check for a file or directory foo accessed by a client process,
+
+* If the user name matches the owner of foo, then the owner permissions are tested;
+* Else if the group of foo matches any of member of the groups list, then the group permissions are tested;
+* Otherwise the other permissions of foo are tested.
+
+If a permissions check fails, the client operation fails.
+
+User Identity
+-------------
+
+As of Hadoop 0.22, Hadoop supports two different modes of operation to determine the user's identity, specified by the hadoop.security.authentication property:
+
+*   **simple**
+
+    In this mode of operation, the identity of a client process is
+    determined by the host operating system. On Unix-like systems,
+    the user name is the equivalent of \`whoami\`.
+
+*   **kerberos**
+
+    In Kerberized operation, the identity of a client process is
+    determined by its Kerberos credentials. For example, in a
+    Kerberized environment, a user may use the kinit utility to
+    obtain a Kerberos ticket-granting-ticket (TGT) and use klist to
+    determine their current principal. When mapping a Kerberos
+    principal to an HDFS username, all components except for the
+    primary are dropped. For example, a principal
+    todd/foobar@CORP.COMPANY.COM will act as the simple username todd on HDFS.
+
+Regardless of the mode of operation, the user identity mechanism is extrinsic to HDFS itself. There is no provision within HDFS for creating user identities, establishing groups, or processing user credentials.
+
+Group Mapping
+-------------
+
+Once a username has been determined as described above, the list of groups is determined by a group mapping service, configured by the hadoop.security.group.mapping property. The default implementation, org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback, will determine if the Java Native Interface (JNI) is available. If JNI is available, the implementation will use the API within hadoop to resolve a list of groups for a user. If JNI is not available then the shell implementation, org.apache.hadoop.security.ShellBasedUnixGroupsMapping, is used. This implementation shells out with the `bash -c groups` command (for a Linux/Unix environment) or the `net group` command (for a Windows environment) to resolve a list of groups for a user.
+
+An alternate implementation, which connects directly to an LDAP server to resolve the list of groups, is available via org.apache.hadoop.security.LdapGroupsMapping. However, this provider should only be used if the required groups reside exclusively in LDAP, and are not materialized on the Unix servers. More information on configuring the group mapping service is available in the Javadocs.
+
+For HDFS, the mapping of users to groups is performed on the NameNode. Thus, the host system configuration of the NameNode determines the group mappings for the users.
+
+Note that HDFS stores the user and group of a file or directory as strings; there is no conversion from user and group identity numbers as is conventional in Unix.
+
+Understanding the Implementation
+--------------------------------
+
+Each file or directory operation passes the full path name to the name node, and the permissions checks are applied along the path for each operation. The client framework will implicitly associate the user identity with the connection to the name node, reducing the need for changes to the existing client API. It has always been the case that when one operation on a file succeeds, the operation might fail when repeated because the file, or some directory on the path, no longer exists. For instance, when the client first begins reading a file, it makes a first request to the name node to discover the location of the first blocks of the file. A second request made to find additional blocks may fail. On the other hand, deleting a file does not revoke access by a client that already knows the blocks of the file. With the addition of permissions, a client's access to a file may be withdrawn between requests. Again, changing permissions does not revoke the access of a client that already 
 knows the file's blocks.
+
+Changes to the File System API
+------------------------------
+
+All methods that use a path parameter will throw `AccessControlException` if permission checking fails.
+
+New methods:
+
+* `public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long
+  blockSize, Progressable progress) throws IOException;`
+* `public boolean mkdirs(Path f, FsPermission permission) throws IOException;`
+* `public void setPermission(Path p, FsPermission permission) throws IOException;`
+* `public void setOwner(Path p, String username, String groupname) throws IOException;`
+* `public FileStatus getFileStatus(Path f) throws IOException;`will additionally return the user, group and mode associated with the path.
+
+The mode of a new file or directory is restricted my the umask set as a configuration parameter. When the existing `create(path, …)` method (without the permission parameter) is used, the mode of the new file is `0666 & ^umask`. When the new `create(path, permission, …)` method (with the permission parameter P) is used, the mode of the new file is `P & ^umask & 0666`. When a new directory is created with the existing `mkdirs(path)` method (without the permission parameter), the mode of the new directory is `0777 & ^umask`. When the new `mkdirs(path, permission)` method (with the permission parameter P) is used, the mode of new directory is `P & ^umask & 0777`.
+
+Changes to the Application Shell
+--------------------------------
+
+New operations:
+
+*   `chmod [-R] mode file ...`
+
+    Only the owner of a file or the super-user is permitted to change the mode of a file.
+
+*   `chgrp [-R] group file ...`
+
+    The user invoking chgrp must belong to the specified group and be
+    the owner of the file, or be the super-user.
+
+*   `chown [-R] [owner][:[group]] file ...`
+
+    The owner of a file may only be altered by a super-user.
+
+*   `ls file ...`
+
+*   `lsr file ...`
+
+    The output is reformatted to display the owner, group and mode.
+
+The Super-User
+--------------
+
+The super-user is the user with the same identity as name node process itself. Loosely, if you started the name node, then you are the super-user. The super-user can do anything in that permissions checks never fail for the super-user. There is no persistent notion of who was the super-user; when the name node is started the process identity determines who is the super-user for now. The HDFS super-user does not have to be the super-user of the name node host, nor is it necessary that all clusters have the same super-user. Also, an experimenter running HDFS on a personal workstation, conveniently becomes that installation's super-user without any configuration.
+
+In addition, the administrator my identify a distinguished group using a configuration parameter. If set, members of this group are also super-users.
+
+The Web Server
+--------------
+
+By default, the identity of the web server is a configuration parameter. That is, the name node has no notion of the identity of the real user, but the web server behaves as if it has the identity (user and groups) of a user chosen by the administrator. Unless the chosen identity matches the super-user, parts of the name space may be inaccessible to the web server.
+
+ACLs (Access Control Lists)
+---------------------------
+
+In addition to the traditional POSIX permissions model, HDFS also supports POSIX ACLs (Access Control Lists). ACLs are useful for implementing permission requirements that differ from the natural organizational hierarchy of users and groups. An ACL provides a way to set different permissions for specific named users or named groups, not only the file's owner and the file's group.
+
+By default, support for ACLs is disabled, and the NameNode disallows creation of ACLs. To enable support for ACLs, set `dfs.namenode.acls.enabled` to true in the NameNode configuration.
+
+An ACL consists of a set of ACL entries. Each ACL entry names a specific user or group and grants or denies read, write and execute permissions for that specific user or group. For example:
+
+       user::rw-
+       user:bruce:rwx                  #effective:r--
+       group::r-x                      #effective:r--
+       group:sales:rwx                 #effective:r--
+       mask::r--
+       other::r--
+
+ACL entries consist of a type, an optional name and a permission string. For display purposes, ':' is used as the delimiter between each field. In this example ACL, the file owner has read-write access, the file group has read-execute access and others have read access. So far, this is equivalent to setting the file's permission bits to 654.
+
+Additionally, there are 2 extended ACL entries for the named user bruce and the named group sales, both granted full access. The mask is a special ACL entry that filters the permissions granted to all named user entries and named group entries, and also the unnamed group entry. In the example, the mask has only read permissions, and we can see that the effective permissions of several ACL entries have been filtered accordingly.
+
+Every ACL must have a mask. If the user doesn't supply a mask while setting an ACL, then a mask is inserted automatically by calculating the union of permissions on all entries that would be filtered by the mask.
+
+Running `chmod` on a file that has an ACL actually changes the permissions of the mask. Since the mask acts as a filter, this effectively constrains the permissions of all extended ACL entries instead of changing just the group entry and possibly missing other extended ACL entries.
+
+The model also differentiates between an "access ACL", which defines the rules to enforce during permission checks, and a "default ACL", which defines the ACL entries that new child files or sub-directories receive automatically during creation. For example:
+
+       user::rwx
+       group::r-x
+       other::r-x
+       default:user::rwx
+       default:user:bruce:rwx          #effective:r-x
+       default:group::r-x
+       default:group:sales:rwx         #effective:r-x
+       default:mask::r-x
+       default:other::r-x
+
+Only directories may have a default ACL. When a new file or sub-directory is created, it automatically copies the default ACL of its parent into its own access ACL. A new sub-directory also copies it to its own default ACL. In this way, the default ACL will be copied down through arbitrarily deep levels of the file system tree as new sub-directories get created.
+
+The exact permission values in the new child's access ACL are subject to filtering by the mode parameter. Considering the default umask of 022, this is typically 755 for new directories and 644 for new files. The mode parameter filters the copied permission values for the unnamed user (file owner), the mask and other. Using this particular example ACL, and creating a new sub-directory with 755 for the mode, this mode filtering has no effect on the final result. However, if we consider creation of a file with 644 for the mode, then mode filtering causes the new file's ACL to receive read-write for the unnamed user (file owner), read for the mask and read for others. This mask also means that effective permissions for named user bruce and named group sales are only read.
+
+Note that the copy occurs at time of creation of the new file or sub-directory. Subsequent changes to the parent's default ACL do not change existing children.
+
+The default ACL must have all minimum required ACL entries, including the unnamed user (file owner), unnamed group (file group) and other entries. If the user doesn't supply one of these entries while setting a default ACL, then the entries are inserted automatically by copying the corresponding permissions from the access ACL, or permission bits if there is no access ACL. The default ACL also must have mask. As described above, if the mask is unspecified, then a mask is inserted automatically by calculating the union of permissions on all entries that would be filtered by the mask.
+
+When considering a file that has an ACL, the algorithm for permission checks changes to:
+
+* If the user name matches the owner of file, then the owner
+  permissions are tested;
+
+* Else if the user name matches the name in one of the named user entries,
+  then these permissions are tested, filtered by the mask permissions;
+
+* Else if the group of file matches any member of the groups list,
+  and if these permissions filtered by the mask grant access, then these
+  permissions are used;
+
+* Else if there is a named group entry matching a member of the groups list,
+  and if these permissions filtered by the mask grant access, then these
+  permissions are used;
+
+* Else if the file group or any named group entry matches a member of the
+  groups list, but access was not granted by any of those permissions, then
+  access is denied;
+
+* Otherwise the other permissions of file are tested.
+
+Best practice is to rely on traditional permission bits to implement most permission requirements, and define a smaller number of ACLs to augment the permission bits with a few exceptional rules. A file with an ACL incurs an additional cost in memory in the NameNode compared to a file that has only permission bits.
+
+ACLs File System API
+--------------------
+
+New methods:
+
+* `public void modifyAclEntries(Path path, List<AclEntry> aclSpec) throws IOException;`
+* `public void removeAclEntries(Path path, List<AclEntry> aclSpec) throws IOException;`
+* `public void public void removeDefaultAcl(Path path) throws IOException;`
+* `public void removeAcl(Path path) throws IOException;`
+* `public void setAcl(Path path, List<AclEntry> aclSpec) throws IOException;`
+* `public AclStatus getAclStatus(Path path) throws IOException;`
+
+ACLs Shell Commands
+-------------------
+
+*   `hdfs dfs -getfacl [-R] <path>`
+
+    Displays the Access Control Lists (ACLs) of files and directories. If a
+    directory has a default ACL, then getfacl also displays the default ACL.
+
+*   `hdfs dfs -setfacl [-R] [-b |-k -m |-x <acl_spec> <path>] |[--set <acl_spec> <path>] `
+
+    Sets Access Control Lists (ACLs) of files and directories.
+
+*   `hdfs dfs -ls <args>`
+
+    The output of `ls` will append a '+' character to the permissions
+    string of any file or directory that has an ACL.
+
+    See the [File System Shell](../hadoop-common/FileSystemShell.html)
+    documentation for full coverage of these commands.
+
+Configuration Parameters
+------------------------
+
+*   `dfs.permissions.enabled = true`
+
+    If yes use the permissions system as described here. If no,
+    permission checking is turned off, but all other behavior is
+    unchanged. Switching from one parameter value to the other does not
+    change the mode, owner or group of files or directories.
+    Regardless of whether permissions are on or off, chmod, chgrp, chown and
+    setfacl always check permissions. These functions are only useful in
+    the permissions context, and so there is no backwards compatibility
+    issue. Furthermore, this allows administrators to reliably set owners and permissions
+    in advance of turning on regular permissions checking.
+
+*   `dfs.web.ugi = webuser,webgroup`
+
+    The user name to be used by the web server. Setting this to the
+    name of the super-user allows any web client to see everything.
+    Changing this to an otherwise unused identity allows web clients to
+    see only those things visible using "other" permissions. Additional
+    groups may be added to the comma-separated list.
+
+*   `dfs.permissions.superusergroup = supergroup`
+
+    The name of the group of super-users.
+
+*   `fs.permissions.umask-mode = 0022`
+
+    The umask used when creating files and directories. For
+    configuration files, the decimal value 18 may be used.
+
+*   `dfs.cluster.administrators = ACL-for-admins`
+
+    The administrators for the cluster specified as an ACL. This
+    controls who can access the default servlets, etc. in the HDFS.
+
+*   `dfs.namenode.acls.enabled = true`
+
+    Set to true to enable support for HDFS ACLs (Access Control Lists). By
+    default, ACLs are disabled. When ACLs are disabled, the NameNode rejects
+    all attempts to set an ACL.
+
+