Browse Source

HDFS-10659. Namenode crashes after Journalnode re-installation in an HA cluster due to missing paxos directory. Contributed by star, Hanisha Koneru.

Signed-off-by: Wei-Chiu Chuang <weichiu@apache.org>
Hanisha Koneru 6 years ago
parent
commit
7217494f40

+ 17 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java

@@ -82,7 +82,8 @@ class JNStorage extends Storage {
   }
 
   @Override
-  public boolean isPreUpgradableLayout(StorageDirectory sd) throws IOException {
+  public boolean isPreUpgradableLayout(StorageDirectory sd)
+      throws IOException {
     return false;
   }
 
@@ -90,7 +91,8 @@ class JNStorage extends Storage {
    * Find an edits file spanning the given transaction ID range.
    * If no such file exists, an exception is thrown.
    */
-  File findFinalizedEditsFile(long startTxId, long endTxId) throws IOException {
+  File findFinalizedEditsFile(long startTxId, long endTxId)
+      throws IOException {
     File ret = new File(sd.getCurrentDir(),
         NNStorage.getFinalizedEditsFileName(startTxId, endTxId));
     if (!ret.exists()) {
@@ -152,11 +154,18 @@ class JNStorage extends Storage {
    * paxos-like recovery process for the given log segment.
    */
   File getPaxosFile(long segmentTxId) {
-    return new File(getPaxosDir(), String.valueOf(segmentTxId));
+    return new File(getOrCreatePaxosDir(), String.valueOf(segmentTxId));
   }
   
-  File getPaxosDir() {
-    return new File(sd.getCurrentDir(), "paxos");
+  File getOrCreatePaxosDir() {
+    File paxosDir = new File(sd.getCurrentDir(), "paxos");
+    if(!paxosDir.exists()) {
+      LOG.info("Creating paxos dir: {}", paxosDir.toPath());
+      if(!paxosDir.mkdir()) {
+        LOG.error("Could not create paxos dir: {}", paxosDir.toPath());
+      }
+    }
+    return paxosDir;
   }
   
   File getRoot() {
@@ -170,7 +179,8 @@ class JNStorage extends Storage {
   void purgeDataOlderThan(long minTxIdToKeep) throws IOException {
     purgeMatching(sd.getCurrentDir(),
         CURRENT_DIR_PURGE_REGEXES, minTxIdToKeep);
-    purgeMatching(getPaxosDir(), PAXOS_DIR_PURGE_REGEXES, minTxIdToKeep);
+    purgeMatching(getOrCreatePaxosDir(),
+        PAXOS_DIR_PURGE_REGEXES, minTxIdToKeep);
   }
   
   /**
@@ -221,16 +231,10 @@ class JNStorage extends Storage {
     unlockAll();
     sd.clearDirectory();
     writeProperties(sd);
-    createPaxosDir();
+    getOrCreatePaxosDir();
     analyzeStorage();
   }
   
-  void createPaxosDir() throws IOException {
-    if (!getPaxosDir().mkdirs()) {
-      throw new IOException("Could not create paxos dir: " + getPaxosDir());
-    }
-  }
-  
   void analyzeStorage() throws IOException {
     this.state = sd.analyzeStorage(StartupOption.REGULAR, this);
     if (state == StorageState.NORMAL) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java

@@ -1119,7 +1119,7 @@ public class Journal implements Closeable {
         + ".\n   new LV = " + storage.getLayoutVersion()
         + "; new CTime = " + storage.getCTime());
     storage.getJournalManager().doUpgrade(storage);
-    storage.createPaxosDir();
+    storage.getOrCreatePaxosDir();
     
     // Copy over the contents of the epoch data files to the new dir.
     File currentDir = storage.getSingularStorageDir().getCurrentDir();