You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by ha...@apache.org on 2010/11/24 23:57:12 UTC
svn commit: r1038859 - in /hadoop/hdfs/trunk: CHANGES.txt
src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
Author: hairong
Date: Wed Nov 24 22:57:11 2010
New Revision: 1038859
URL: http://svn.apache.org/viewvc?rev=1038859&view=rev
Log:
HDFS-1458. Improve checkpoint performance by avoiding unnecessary image downloads and loading. Contributed by Hairong Kuang.
Modified:
hadoop/hdfs/trunk/CHANGES.txt
hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
Modified: hadoop/hdfs/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/CHANGES.txt?rev=1038859&r1=1038858&r2=1038859&view=diff
==============================================================================
--- hadoop/hdfs/trunk/CHANGES.txt (original)
+++ hadoop/hdfs/trunk/CHANGES.txt Wed Nov 24 22:57:11 2010
@@ -17,6 +17,9 @@ Trunk (unreleased changes)
OPTIMIZATIONS
+ HDFS-1458. Improve checkpoint performance by avoiding unnecessary image
+ downloads and loading. (hairong)
+
BUG FIXES
HDFS-1516. mvn-install is broken after 0.22 branch creation. (cos)
Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java?rev=1038859&r1=1038858&r2=1038859&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java Wed Nov 24 22:57:11 2010
@@ -312,6 +312,7 @@ public class SecondaryNameNode implement
LOG.error("Exception in doCheckpoint: ");
LOG.error(StringUtils.stringifyException(e));
e.printStackTrace();
+ checkpointImage.imageDigest = null;
} catch (Throwable e) {
LOG.error("Throwable Exception in doCheckpoint: ");
LOG.error(StringUtils.stringifyException(e));
@@ -324,28 +325,39 @@ public class SecondaryNameNode implement
/**
* Download <code>fsimage</code> and <code>edits</code>
* files from the name-node.
+ * @return true if a new image has been downloaded and needs to be loaded
* @throws IOException
*/
- private void downloadCheckpointFiles(final CheckpointSignature sig
+ private boolean downloadCheckpointFiles(final CheckpointSignature sig
) throws IOException {
try {
- UserGroupInformation.getCurrentUser().doAs(new PrivilegedExceptionAction<Void>() {
+ Boolean b = UserGroupInformation.getCurrentUser().doAs(
+ new PrivilegedExceptionAction<Boolean>() {
@Override
- public Void run() throws Exception {
+ public Boolean run() throws Exception {
checkpointImage.cTime = sig.cTime;
checkpointImage.checkpointTime = sig.checkpointTime;
- checkpointImage.imageDigest = sig.imageDigest;
-
+
// get fsimage
- String fileid = "getimage=1";
- Collection<File> list = checkpointImage.getFiles(NameNodeFile.IMAGE,
- NameNodeDirType.IMAGE);
- File[] srcNames = list.toArray(new File[list.size()]);
- assert srcNames.length > 0 : "No checkpoint targets.";
- TransferFsImage.getFileClient(fsName, fileid, srcNames, false);
- LOG.info("Downloaded file " + srcNames[0].getName() + " size " +
- srcNames[0].length() + " bytes.");
+ String fileid;
+ Collection<File> list;
+ File[] srcNames;
+ boolean downloadImage = true;
+ if (sig.imageDigest.equals(checkpointImage.imageDigest)) {
+ downloadImage = false;
+ LOG.info("Image has not changed. Will not download image.");
+ } else {
+ fileid = "getimage=1";
+ list = checkpointImage.getFiles(NameNodeFile.IMAGE,
+ NameNodeDirType.IMAGE);
+ srcNames = list.toArray(new File[list.size()]);
+ assert srcNames.length > 0 : "No checkpoint targets.";
+ TransferFsImage.getFileClient(fsName, fileid, srcNames, false);
+ checkpointImage.imageDigest = sig.imageDigest;
+ LOG.info("Downloaded file " + srcNames[0].getName() + " size " +
+ srcNames[0].length() + " bytes.");
+ }
// get edits file
fileid = "getedit=1";
@@ -357,9 +369,10 @@ public class SecondaryNameNode implement
srcNames[0].length() + " bytes.");
checkpointImage.checkpointUploadDone();
- return null;
+ return Boolean.valueOf(downloadImage);
}
});
+ return b.booleanValue();
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
@@ -404,8 +417,9 @@ public class SecondaryNameNode implement
/**
* Create a new checkpoint
+ * @return if the image is fetched from primary or not
*/
- void doCheckpoint() throws IOException {
+ boolean doCheckpoint() throws IOException {
// Do the required initialization of the merge work area.
startCheckpoint();
@@ -420,8 +434,8 @@ public class SecondaryNameNode implement
"after creating edits.new");
}
- downloadCheckpointFiles(sig); // Fetch fsimage and edits
- doMerge(sig); // Do the merge
+ boolean loadImage = downloadCheckpointFiles(sig); // Fetch fsimage and edits
+ doMerge(sig, loadImage); // Do the merge
//
// Upload the new image into the NameNode. Then tell the Namenode
@@ -440,6 +454,8 @@ public class SecondaryNameNode implement
LOG.warn("Checkpoint done. New Image Size: "
+ checkpointImage.getFsImageName().length());
+
+ return loadImage;
}
private void startCheckpoint() throws IOException {
@@ -453,11 +469,12 @@ public class SecondaryNameNode implement
* Merge downloaded image and edits and write the new image into
* current storage directory.
*/
- private void doMerge(CheckpointSignature sig) throws IOException {
+ private void doMerge(CheckpointSignature sig, boolean loadImage)
+ throws IOException {
FSNamesystem namesystem =
new FSNamesystem(checkpointImage, conf);
assert namesystem.dir.fsImage == checkpointImage;
- checkpointImage.doMerge(sig);
+ checkpointImage.doMerge(sig, loadImage);
}
/**
@@ -661,21 +678,29 @@ public class SecondaryNameNode implement
/**
* Merge image and edits, and verify consistency with the signature.
*/
- private void doMerge(CheckpointSignature sig) throws IOException {
+ private void doMerge(CheckpointSignature sig, boolean loadImage)
+ throws IOException {
getEditLog().open();
StorageDirectory sdName = null;
StorageDirectory sdEdits = null;
Iterator<StorageDirectory> it = null;
- it = dirIterator(NameNodeDirType.IMAGE);
- if (it.hasNext())
- sdName = it.next();
+ if (loadImage) {
+ it = dirIterator(NameNodeDirType.IMAGE);
+ if (it.hasNext())
+ sdName = it.next();
+ if (sdName == null) {
+ throw new IOException("Could not locate checkpoint fsimage");
+ }
+ }
it = dirIterator(NameNodeDirType.EDITS);
if (it.hasNext())
sdEdits = it.next();
- if ((sdName == null) || (sdEdits == null))
- throw new IOException("Could not locate checkpoint directories");
- this.layoutVersion = -1; // to avoid assert in loadFSImage()
- loadFSImage(FSImage.getImageFile(sdName, NameNodeFile.IMAGE));
+ if (sdEdits == null)
+ throw new IOException("Could not locate checkpoint edits");
+ if (loadImage) {
+ this.layoutVersion = -1; // to avoid assert in loadFSImage()
+ loadFSImage(FSImage.getImageFile(sdName, NameNodeFile.IMAGE));
+ }
loadFSEdits(sdEdits);
sig.validateStorageInfo(this);
saveNamespace(false);
Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java?rev=1038859&r1=1038858&r2=1038859&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java Wed Nov 24 22:57:11 2010
@@ -789,4 +789,56 @@ public class TestCheckpoint extends Test
if(cluster!= null) cluster.shutdown();
}
}
+
+ /**
+ * Simulate a secondary node failure to transfer image
+ * back to the name-node.
+ * Used to truncate primary fsimage file.
+ */
+ @SuppressWarnings("deprecation")
+ public void testSecondaryImageDownload(Configuration conf)
+ throws IOException {
+ System.out.println("Starting testSecondaryImageDownload");
+ conf.set(DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY, "0.0.0.0:0");
+ Path dir = new Path("/checkpoint");
+ MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+ .numDataNodes(numDatanodes)
+ .format(false).build();
+ cluster.waitActive();
+ FileSystem fileSys = cluster.getFileSystem();
+ FSImage image = cluster.getNameNode().getFSImage();
+ try {
+ assertTrue(!fileSys.exists(dir));
+ //
+ // Make the checkpoint
+ //
+ SecondaryNameNode secondary = startSecondaryNameNode(conf);
+ long fsimageLength = FSImage.getImageFile(
+ image.dirIterator(NameNodeDirType.IMAGE).next(),
+ NameNodeFile.IMAGE).length();
+ assertFalse("Image is downloaded", secondary.doCheckpoint());
+
+ // Verify that image file sizes did not change.
+ for (Iterator<StorageDirectory> it =
+ image.dirIterator(NameNodeDirType.IMAGE); it.hasNext();) {
+ assertTrue("Image size does not change", FSImage.getImageFile(it.next(),
+ NameNodeFile.IMAGE).length() == fsimageLength);
+ }
+
+ // change namespace
+ fileSys.mkdirs(dir);
+ assertTrue("Image is not downloaded", secondary.doCheckpoint());
+
+ for (Iterator<StorageDirectory> it =
+ image.dirIterator(NameNodeDirType.IMAGE); it.hasNext();) {
+ assertTrue("Image size increased", FSImage.getImageFile(it.next(),
+ NameNodeFile.IMAGE).length() > fsimageLength);
+ }
+
+ secondary.shutdown();
+ } finally {
+ fileSys.close();
+ cluster.shutdown();
+ }
+ }
}