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
This commit is contained in:
parent
9b81883952
commit
e9a17c8ce0
|
@ -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)
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
Loading…
Reference in New Issue