HDFS-3236. NameNode does not initialize generic conf keys when started with -initializeSharedEditsDir. Contributed by Aaron T. Myers.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1311555 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
72c44aa50e
commit
e10e3c0c20
|
@ -347,6 +347,9 @@ Release 2.0.0 - UNRELEASED
|
|||
|
||||
HDFS-3234. Accidentally left log message in GetConf after HDFS-3226 (todd)
|
||||
|
||||
HDFS-3236. NameNode does not initialize generic conf keys when started
|
||||
with -initializeSharedEditsDir (atm)
|
||||
|
||||
BREAKDOWN OF HDFS-1623 SUBTASKS
|
||||
|
||||
HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)
|
||||
|
|
|
@ -760,6 +760,9 @@ public class NameNode {
|
|||
*/
|
||||
private static boolean initializeSharedEdits(Configuration conf,
|
||||
boolean force, boolean interactive) {
|
||||
String nsId = DFSUtil.getNamenodeNameServiceId(conf);
|
||||
String namenodeId = HAUtil.getNameNodeId(conf, nsId);
|
||||
initializeGenericKeys(conf, nsId, namenodeId);
|
||||
NNStorage existingStorage = null;
|
||||
try {
|
||||
FSNamesystem fsns = FSNamesystem.loadFromDisk(conf,
|
||||
|
|
|
@ -25,6 +25,8 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||
|
@ -117,4 +119,17 @@ public class TestInitializeSharedEdits {
|
|||
assertFalse(NameNode.initializeSharedEdits(conf, false));
|
||||
assertTrue(NameNode.initializeSharedEdits(conf, false));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInitializeSharedEditsConfiguresGenericConfKeys() {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(DFSConfigKeys.DFS_FEDERATION_NAMESERVICES, "ns1");
|
||||
conf.set(DFSUtil.addKeySuffixes(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX,
|
||||
"ns1"), "nn1,nn2");
|
||||
conf.set(DFSUtil.addKeySuffixes(DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY,
|
||||
"ns1", "nn1"), "localhost:1234");
|
||||
assertNull(conf.get(DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY));
|
||||
NameNode.initializeSharedEdits(conf);
|
||||
assertNotNull(conf.get(DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY));
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue