浏览代码

HDFS-3389. Document the BKJM usage in Namenode HA. Contributed by Uma Maheswara Rao G and Ivan Kelly.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1349466 13f79535-47bb-0310-9956-ffa450edef68
Uma Maheswara Rao G 13 年之前
父节点
当前提交
f9c938666b

+ 152 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/HDFSHighAvailability.apt.vm

@@ -712,4 +712,155 @@ digest:hdfs-zkfcs:vlUvLnd8MlacsE80rDuu6ONESbM=:rwcda
 
 
   Even if automatic failover is configured, you may initiate a manual failover
   Even if automatic failover is configured, you may initiate a manual failover
   using the same <<<hdfs haadmin>>> command. It will perform a coordinated
   using the same <<<hdfs haadmin>>> command. It will perform a coordinated
-  failover.
+  failover.
+
+ 
+* BookKeeper as a Shared storage (EXPERIMENTAL)
+
+   One option for shared storage for the NameNode is BookKeeper. 
+  BookKeeper achieves high availability and strong durability guarantees by replicating
+  edit log entries across multiple storage nodes. The edit log can be striped across 
+  the storage nodes for high performance. Fencing is supported in the protocol, i.e, 
+  BookKeeper will not allow two writers to write the single edit log.
+
+  The meta data for BookKeeper is stored in ZooKeeper.
+  In current HA architecture, a Zookeeper cluster is required for ZKFC. The same cluster can be
+  for BookKeeper metadata.
+
+  For more details on building a BookKeeper cluster, please refer to the 
+   {{{http://zookeeper.apache.org/bookkeeper/docs/trunk/bookkeeperConfig.html }BookKeeper documentation}}
+
+ The BookKeeperJournalManager is an implementation of the HDFS JournalManager interface, which allows custom write ahead logging implementations to be plugged into the HDFS NameNode.
+ 
+ **<<BookKeeper Journal Manager>>
+
+   To use BookKeeperJournalManager, add the following to hdfs-site.xml.
+
+----
+    <property>
+      <name>dfs.namenode.shared.edits.dir</name>
+      <value>bookkeeper://zk1:2181;zk2:2181;zk3:2181/hdfsjournal</value>
+    </property>
+
+    <property>
+      <name>dfs.namenode.edits.journal-plugin.bookkeeper</name>
+      <value>org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager</value>
+    </property>
+----
+
+   The URI format for bookkeeper is <<<bookkeeper://[zkEnsemble]/[rootZnode]
+   [zookkeeper ensemble]>>> is a list of semi-colon separated, zookeeper host:port
+   pairs. In the example above there are 3 servers, in the ensemble,
+   zk1, zk2 & zk3, each one listening on port 2181.
+
+   <<<[root znode]>>> is the path of the zookeeper znode, under which the edit log
+   information will be stored.
+
+   The class specified for the journal-plugin must be available in the NameNode's
+   classpath. We explain how to generate a jar file with the journal manager and
+   its dependencies, and how to put it into the classpath below.
+
+ *** <<More configuration options>> 
+
+     * <<dfs.namenode.bookkeeperjournal.output-buffer-size>> - 
+       Number of bytes a bookkeeper journal stream will buffer before
+       forcing a flush. Default is 1024.
+     
+----
+       <property>
+         <name>dfs.namenode.bookkeeperjournal.output-buffer-size</name>
+         <value>1024</value>
+       </property>
+----
+
+     * <<dfs.namenode.bookkeeperjournal.ensemble-size>> - 
+       Number of bookkeeper servers in edit log ensembles. This
+       is the number of bookkeeper servers which need to be available
+       for the edit log to be writable. Default is 3.
+
+----
+       <property>
+         <name>dfs.namenode.bookkeeperjournal.ensemble-size</name>
+         <value>3</value>
+       </property>
+----
+
+     * <<dfs.namenode.bookkeeperjournal.quorum-size>> - 
+       Number of bookkeeper servers in the write quorum. This is the
+       number of bookkeeper servers which must have acknowledged the
+       write of an entry before it is considered written. Default is 2.
+
+----
+       <property>
+         <name>dfs.namenode.bookkeeperjournal.quorum-size</name>
+         <value>2</value>
+       </property>
+----
+
+     * <<dfs.namenode.bookkeeperjournal.digestPw>> - 
+       Password to use when creating edit log segments.
+
+----
+       <property>
+        <name>dfs.namenode.bookkeeperjournal.digestPw</name>
+        <value>myPassword</value>
+       </property>
+----
+
+     * <<dfs.namenode.bookkeeperjournal.zk.session.timeout>> - 
+       Session timeout for Zookeeper client from BookKeeper Journal Manager.
+       Hadoop recommends that this value should be less than the ZKFC 
+       session timeout value. Default value is 3000.
+
+----
+       <property>
+         <name>dfs.namenode.bookkeeperjournal.zk.session.timeout</name>
+         <value>3000</value>
+       </property>
+----
+
+ *** <<Building BookKeeper Journal Manager plugin jar>>
+
+     To generate the distribution packages for BK journal, do the
+     following.
+
+     $ mvn clean package -Pdist
+
+     This will generate a jar with the BookKeeperJournalManager, all the dependencies
+     needed by the journal manager,
+     hadoop-hdfs/src/contrib/bkjournal/target/hadoop-hdfs-bkjournal-<VERSION>.jar
+
+     Note that the -Pdist part of the build command is important, as otherwise
+     the dependencies would not be packaged in the jar. The dependencies included in
+     the jar are {{{http://maven.apache.org/plugins/maven-shade-plugin/}shaded}} to
+     avoid conflicts with other dependencies of the NameNode.
+
+ *** <<Putting the BookKeeperJournalManager in the NameNode classpath>>
+
+    To run a HDFS namenode using BookKeeper as a backend, copy the bkjournal
+    jar, generated above, into the lib directory of hdfs. In the standard 
+    distribution of HDFS, this is at $HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/
+
+    cp hadoop-hdfs/src/contrib/bkjournal/target/hadoop-hdfs-bkjournal-<VERSION>.jar $HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/
+
+ *** <<Current limitations>> 
+
+      1) NameNode format command will not format the BookKeeper data automatically. 
+         We have to clean the data manually from BookKeeper cluster 
+         and create the /ledgers/available path in Zookeeper. 
+----
+$ zkCli.sh create /ledgers 0
+$ zkCli.sh create /ledgers/available 0
+----
+         Note:
+          bookkeeper://zk1:2181;zk2:2181;zk3:2181/hdfsjournal
+          The final part /hdfsjournal specifies the znode in zookeeper where
+          ledger metadata will be stored. Administrators may set this to anything
+          they wish.
+
+      2) Security in BookKeeper. BookKeeper does not support SASL nor SSL for
+         connections between the NameNode and BookKeeper storage nodes.
+
+      3) Auto-Recovery of storage node failures. Work inprogress 
+      {{{https://issues.apache.org/jira/browse/BOOKKEEPER-237 }BOOKKEEPER-237}}.
+         Currently we have the tools to manually recover the data from failed storage nodes.