Bladeren bron

HDFS-3573. Supply NamespaceInfo when instantiating JournalManagers. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1356388 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 13 jaren geleden
bovenliggende
commit
7accbabdee

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -102,6 +102,8 @@ Trunk (unreleased changes)
     HDFS-3510.  Editlog pre-allocation is performed prior to writing edits
     to avoid partial edits case disk out of space.(Colin McCabe via suresh)
 
+    HDFS-3573. Supply NamespaceInfo when instantiating JournalManagers (todd)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 11 - 2
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java

@@ -22,6 +22,7 @@ import org.apache.hadoop.hdfs.server.namenode.JournalManager;
 import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.conf.Configuration;
 
 import org.apache.bookkeeper.conf.ClientConfiguration;
@@ -158,11 +159,19 @@ public class BookKeeperJournalManager implements JournalManager {
       (byte)(i) };
   }
 
+  BookKeeperJournalManager(Configuration conf, URI uri) throws IOException {
+    this(conf, uri, null);
+    // TODO(ivank): update BookKeeperJournalManager to do something
+    // with the NamespaceInfo. This constructor has been added
+    // for compatibility with the old tests, and may be removed
+    // when the tests are updated.
+  }
+
   /**
    * Construct a Bookkeeper journal manager.
    */
-  public BookKeeperJournalManager(Configuration conf, URI uri)
-      throws IOException {
+  public BookKeeperJournalManager(Configuration conf, URI uri,
+      NamespaceInfo nsInfo) throws IOException {
     this.conf = conf;
     String zkConnect = uri.getAuthority().replace(";", ",");
     String zkPath = uri.getPath();

+ 9 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -64,6 +64,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
@@ -324,7 +325,7 @@ public class FSEditLog  {
       endCurrentLogSegment(true);
     }
     
-    if (!journalSet.isEmpty()) {
+    if (journalSet != null && !journalSet.isEmpty()) {
       try {
         journalSet.close();
       } catch (IOException ioe) {
@@ -1010,7 +1011,10 @@ public class FSEditLog  {
       minTxIdToKeep <= curSegmentTxId :
       "cannot purge logs older than txid " + minTxIdToKeep +
       " when current segment starts at " + curSegmentTxId;
-
+    if (minTxIdToKeep == 0) {
+      return;
+    }
+    
     // This could be improved to not need synchronization. But currently,
     // journalSet is not threadsafe, so we need to synchronize this method.
     try {
@@ -1260,8 +1264,9 @@ public class FSEditLog  {
 
     try {
       Constructor<? extends JournalManager> cons
-        = clazz.getConstructor(Configuration.class, URI.class);
-      return cons.newInstance(conf, uri);
+        = clazz.getConstructor(Configuration.class, URI.class,
+            NamespaceInfo.class);
+      return cons.newInstance(conf, uri, storage.getNamespaceInfo());
     } catch (Exception e) {
       throw new IllegalArgumentException("Unable to construct journal, "
                                          + uri, e);

+ 14 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -126,12 +126,6 @@ public class FSImage implements Closeable {
     }
 
     this.editLog = new FSEditLog(conf, storage, editsDirs);
-    String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
-    if (!HAUtil.isHAEnabled(conf, nameserviceId)) {
-      editLog.initJournalsForWrite();
-    } else {
-      editLog.initSharedJournalsForRead();
-    }
     
     archivalManager = new NNStorageRetentionManager(conf, storage, editLog);
   }
@@ -511,6 +505,7 @@ public class FSImage implements Closeable {
     // return back the real image
     realImage.getStorage().setStorageInfo(ckptImage.getStorage());
     realImage.getEditLog().setNextTxId(ckptImage.getEditLog().getLastWrittenTxId()+1);
+    realImage.initEditLog();
 
     target.dir.fsImage = realImage;
     realImage.getStorage().setBlockPoolID(ckptImage.getBlockPoolID());
@@ -584,10 +579,8 @@ public class FSImage implements Closeable {
 
     Iterable<EditLogInputStream> editStreams = null;
 
-    if (editLog.isOpenForWrite()) {
-      // We only want to recover streams if we're going into Active mode.
-      editLog.recoverUnclosedStreams();
-    }
+    initEditLog();
+
     if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT, 
                                getLayoutVersion())) {
       // If we're open for write, we're either non-HA or we're the active NN, so
@@ -645,6 +638,17 @@ public class FSImage implements Closeable {
     return needToSave;
   }
 
+  public void initEditLog() {
+    Preconditions.checkState(getNamespaceID() != 0,
+        "Must know namespace ID before initting edit log");
+    String nameserviceId = DFSUtil.getNamenodeNameServiceId(conf);
+    if (!HAUtil.isHAEnabled(conf, nameserviceId)) {
+      editLog.initJournalsForWrite();
+      editLog.recoverUnclosedStreams();
+    } else {
+      editLog.initSharedJournalsForRead();
+    }
+  }
 
   /**
    * @param imageFile the image file that was loaded

+ 7 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java

@@ -228,12 +228,6 @@ class FileJournalManager implements JournalManager {
         (inProgressOk ? " (inProgress ok) " : " (excluding inProgress) ") +
         "from among " + elfs.size() + " candidate file(s)");
     for (EditLogFile elf : elfs) {
-      if (elf.lastTxId < fromTxId) {
-        LOG.debug("passing over " + elf + " because it ends at " +
-            elf.lastTxId + ", but we only care about transactions " +
-            "as new as " + fromTxId);
-        continue;
-      }
       if (elf.isInProgress()) {
         if (!inProgressOk) {
           LOG.debug("passing over " + elf + " because it is in progress " +
@@ -248,6 +242,13 @@ class FileJournalManager implements JournalManager {
           continue;
         }
       }
+      if (elf.lastTxId < fromTxId) {
+        assert elf.lastTxId != HdfsConstants.INVALID_TXID;
+        LOG.debug("passing over " + elf + " because it ends at " +
+            elf.lastTxId + ", but we only care about transactions " +
+            "as new as " + fromTxId);
+        continue;
+      }
       EditLogFileInputStream elfis = new EditLogFileInputStream(elf.getFile(),
             elf.getFirstTxId(), elf.getLastTxId(), elf.isInProgress());
       LOG.debug("selecting edit log stream " + elf);

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java

@@ -1125,4 +1125,13 @@ public class NNStorage extends Storage implements Closeable {
     inspectStorageDirs(inspector);
     return inspector;
   }
+
+  public NamespaceInfo getNamespaceInfo() {
+    return new NamespaceInfo(
+        getNamespaceID(),
+        getClusterID(),
+        getBlockPoolID(),
+        getCTime(),
+        getDistributedUpgradeVersion());
+  }
 }

+ 1 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -67,7 +67,6 @@ import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
-import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.util.AtomicFileOutputStream;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.Server;
@@ -830,12 +829,7 @@ public class NameNode {
           Lists.<URI>newArrayList(),
           sharedEditsDirs);
       
-      newSharedStorage.format(new NamespaceInfo(
-          existingStorage.getNamespaceID(),
-          existingStorage.getClusterID(),
-          existingStorage.getBlockPoolID(),
-          existingStorage.getCTime(),
-          existingStorage.getDistributedUpgradeVersion()));
+      newSharedStorage.format(existingStorage.getNamespaceInfo());
       
       // Need to make sure the edit log segments are in good shape to initialize
       // the shared edits dir.

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java

@@ -188,6 +188,8 @@ public class BootstrapStandby implements Tool, Configurable {
     // Load the newly formatted image, using all of the directories (including shared
     // edits)
     FSImage image = new FSImage(conf);
+    image.getStorage().setStorageInfo(storage);
+    image.initEditLog();
     assert image.getEditLog().isOpenForRead() :
         "Expected edit log to be open for read";
     

+ 22 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGenericJournalConf.java

@@ -24,12 +24,15 @@ import static org.junit.Assert.*;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 
 import java.net.URI;
 import java.util.Collection;
 import java.io.IOException;
 
 public class TestGenericJournalConf {
+  private static final String DUMMY_URI = "dummy://test";
+
   /** 
    * Test that an exception is thrown if a journal class doesn't exist
    * in the configuration 
@@ -114,12 +117,17 @@ public class TestGenericJournalConf {
 
     conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_PLUGIN_PREFIX + ".dummy",
              DummyJournalManager.class.getName());
-    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
-             "dummy://test");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, DUMMY_URI);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKED_VOLUMES_MINIMUM_KEY, 0);
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
       cluster.waitActive();
+      
+      assertNotNull(DummyJournalManager.conf);
+      assertEquals(new URI(DUMMY_URI), DummyJournalManager.uri);
+      assertNotNull(DummyJournalManager.nsInfo);
+      assertEquals(DummyJournalManager.nsInfo.getClusterID(),
+          cluster.getNameNode().getNamesystem().getClusterId());
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -128,7 +136,17 @@ public class TestGenericJournalConf {
   }
 
   public static class DummyJournalManager implements JournalManager {
-    public DummyJournalManager(Configuration conf, URI u) {}
+    static Configuration conf = null;
+    static URI uri = null;
+    static NamespaceInfo nsInfo = null;
+    
+    public DummyJournalManager(Configuration conf, URI u,
+        NamespaceInfo nsInfo) {
+      // Set static vars so the test case can verify them.
+      DummyJournalManager.conf = conf;
+      DummyJournalManager.uri = u;
+      DummyJournalManager.nsInfo = nsInfo; 
+    }
     
     @Override
     public EditLogOutputStream startLogSegment(long txId) throws IOException {
@@ -162,7 +180,7 @@ public class TestGenericJournalConf {
 
   public static class BadConstructorJournalManager extends DummyJournalManager {
     public BadConstructorJournalManager() {
-      super(null, null);
+      super(null, null, null);
     }
   }
 }