Explorar o código

HDFS-6029. Secondary NN fails to checkpoint after -rollingUpgrade prepare. Contributed by Jing Zhao.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5535@1572800 13f79535-47bb-0310-9956-ffa450edef68
Jing Zhao %!s(int64=11) %!d(string=hai) anos
pai
achega
e9a17c8ce0

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES_HDFS-5535.txt

@@ -117,3 +117,6 @@ HDFS-5535 subtasks:
 
     HDFS-6026. Fix TestDFSUpgrade and TestDataNodeRollingUpgrade.
     (jing9 via szetszwo)
+
+    HDFS-6029. Secondary NN fails to checkpoint after -rollingUpgrade prepare.
+    (jing9)

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java

@@ -221,8 +221,8 @@ class Checkpointer extends Daemon {
         LOG.info("Unable to roll forward using only logs. Downloading " +
             "image with txid " + sig.mostRecentCheckpointTxId);
         MD5Hash downloadedHash = TransferFsImage.downloadImageToStorage(
-            backupNode.nnHttpAddress, NameNodeFile.IMAGE,
-            sig.mostRecentCheckpointTxId, bnStorage, true);
+            backupNode.nnHttpAddress, sig.mostRecentCheckpointTxId, bnStorage,
+            true);
         bnImage.saveDigestAndRenameCheckpointImage(NameNodeFile.IMAGE,
             sig.mostRecentCheckpointTxId, downloadedHash);
         lastApplied = sig.mostRecentCheckpointTxId;

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetImageServlet.java

@@ -122,14 +122,14 @@ public class GetImageServlet extends HttpServlet {
         public Void run() throws Exception {
           if (parsedParams.isGetImage()) {
             long txid = parsedParams.getTxId();
-            final NameNodeFile nnf = parsedParams.getNameNodeFile();
             File imageFile = null;
             String errorMessage = "Could not find image";
             if (parsedParams.shouldFetchLatest()) {
               imageFile = nnImage.getStorage().getHighestFsImageName();
             } else {
               errorMessage += " with txid " + txid;
-              imageFile = nnImage.getStorage().getFsImageName(txid, nnf);
+              imageFile = nnImage.getStorage().getFsImage(txid,
+                  EnumSet.of(NameNodeFile.IMAGE, NameNodeFile.IMAGE_ROLLBACK));
             }
             if (imageFile == null) {
               throw new IOException(errorMessage);
@@ -183,7 +183,7 @@ public class GetImageServlet extends HttpServlet {
               // issue a HTTP get request to download the new fsimage 
               MD5Hash downloadImageDigest = TransferFsImage
                   .downloadImageToStorage(parsedParams.getInfoServer(conf),
-                      nnf, txid, nnImage.getStorage(), true);
+                      txid, nnImage.getStorage(), true);
               nnImage.saveDigestAndRenameCheckpointImage(nnf, txid,
                   downloadImageDigest);
               if (nnf == NameNodeFile.IMAGE_ROLLBACK) {
@@ -324,8 +324,10 @@ public class GetImageServlet extends HttpServlet {
 
   static String getParamStringForImage(NameNodeFile nnf, long txid,
       StorageInfo remoteStorageInfo) {
+    final String imageType = nnf == null ? "" : "&" + IMAGE_FILE_TYPE + "="
+        + nnf.name();
     return "getimage=1&" + TXID_PARAM + "=" + txid
-      + "&" + IMAGE_FILE_TYPE + "=" + nnf.name()
+      + imageType
       + "&" + STORAGEINFO_PARAM + "=" +
       remoteStorageInfo.toColonSeparatedString();
   }

+ 19 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java

@@ -512,7 +512,25 @@ public class NNStorage extends Storage implements Closeable,
       }
     }
     return null;
-  }  
+  }
+
+  /**
+   * @return The first image file whose txid is the same with the given txid and
+   * image type is one of the given types.
+   */
+  public File getFsImage(long txid, EnumSet<NameNodeFile> nnfs) {
+    for (Iterator<StorageDirectory> it = dirIterator(NameNodeDirType.IMAGE);
+        it.hasNext();) {
+      StorageDirectory sd = it.next();
+      for (NameNodeFile nnf : nnfs) {
+        File fsImage = getStorageFile(sd, nnf, txid);
+        if (FileUtil.canRead(sd.getRoot()) && fsImage.exists()) {
+          return fsImage;
+        }
+      }
+    }
+    return null;
+  }
 
   public File getFsImageName(long txid) {
     return getFsImageName(txid, NameNodeFile.IMAGE);

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

@@ -445,7 +445,7 @@ public class SecondaryNameNode implements Runnable {
             } else {
               LOG.info("Image has changed. Downloading updated image from NN.");
               MD5Hash downloadedHash = TransferFsImage.downloadImageToStorage(
-                  nnHostPort, NameNodeFile.IMAGE, sig.mostRecentCheckpointTxId,
+                  nnHostPort, sig.mostRecentCheckpointTxId,
                   dstImage.getStorage(), true);
               dstImage.saveDigestAndRenameCheckpointImage(NameNodeFile.IMAGE,
                   sig.mostRecentCheckpointTxId, downloadedHash);

+ 3 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java

@@ -87,10 +87,9 @@ public class TransferFsImage {
         null, false);
   }
 
-  public static MD5Hash downloadImageToStorage(URL fsName, NameNodeFile nnf,
-      long imageTxId, Storage dstStorage, boolean needDigest)
-      throws IOException {
-    String fileid = GetImageServlet.getParamStringForImage(nnf,
+  public static MD5Hash downloadImageToStorage(URL fsName, long imageTxId,
+      Storage dstStorage, boolean needDigest) throws IOException {
+    String fileid = GetImageServlet.getParamStringForImage(null,
         imageTxId, dstStorage);
     String fileName = NNStorage.getCheckpointImageFileName(imageTxId);
     

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

@@ -208,8 +208,7 @@ public class BootstrapStandby implements Tool, Configurable {
 
       // Download that checkpoint into our storage directories.
       MD5Hash hash = TransferFsImage.downloadImageToStorage(
-        otherHttpAddr, NameNodeFile.IMAGE, imageTxId,
-        storage, true);
+        otherHttpAddr, imageTxId, storage, true);
       image.saveDigestAndRenameCheckpointImage(NameNodeFile.IMAGE, imageTxId,
           hash);
     } catch (IOException ioe) {

+ 45 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java

@@ -35,7 +35,9 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
+import org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
+import org.apache.hadoop.io.IOUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -492,4 +494,47 @@ public class TestRollingUpgrade {
       Assert.fail("Query return false");
     }
   }
+
+  /**
+   * In non-HA setup, after rolling upgrade prepare, the Secondary NN should
+   * still be able to do checkpoint
+   */
+  @Test
+  public void testCheckpointWithSNN() throws Exception {
+    MiniDFSCluster cluster = null;
+    DistributedFileSystem dfs = null;
+    SecondaryNameNode snn = null;
+
+    try {
+      Configuration conf = new HdfsConfiguration();
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+
+      conf.set(DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
+          "0.0.0.0:0");
+      snn = new SecondaryNameNode(conf);
+
+      dfs = cluster.getFileSystem();
+      dfs.mkdirs(new Path("/test/foo"));
+
+      snn.doCheckpoint();
+
+      //start rolling upgrade
+      dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      dfs.rollingUpgrade(RollingUpgradeAction.PREPARE);
+      dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+
+      dfs.mkdirs(new Path("/test/bar"));
+      // do checkpoint in SNN again
+      snn.doCheckpoint();
+    } finally {
+      IOUtils.cleanup(null, dfs);
+      if (snn != null) {
+        snn.shutdown();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -1980,8 +1980,7 @@ public class TestCheckpoint {
         .when(dstImage).toColonSeparatedString();
 
       try {
-        TransferFsImage.downloadImageToStorage(fsName, NameNodeFile.IMAGE, 0,
-            dstImage, false);
+        TransferFsImage.downloadImageToStorage(fsName, 0, dstImage, false);
         fail("Storage info was not verified");
       } catch (IOException ioe) {
         String msg = StringUtils.stringifyException(ioe);