|
@@ -712,4 +712,155 @@ digest:hdfs-zkfcs:vlUvLnd8MlacsE80rDuu6ONESbM=:rwcda
|
|
|
|
|
|
Even if automatic failover is configured, you may initiate a manual failover
|
|
|
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.
|