You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by dh...@apache.org on 2007/09/24 22:57:23 UTC
svn commit: r578964 - in /lucene/hadoop/trunk: ./
src/java/org/apache/hadoop/dfs/ src/test/org/apache/hadoop/dfs/
Author: dhruba
Date: Mon Sep 24 13:57:20 2007
New Revision: 578964
URL: http://svn.apache.org/viewvc?rev=578964&view=rev
Log:
HADOOP-1076. Periodic checkpoint can continue even if an earlier
checkpoint encountered an error. (Dhruba Borthakur via dhruba)
Modified:
lucene/hadoop/trunk/CHANGES.txt
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSEditLog.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/GetImageServlet.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/SecondaryNameNode.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/TransferFsImage.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/dfs/TestCheckpoint.java
Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?rev=578964&r1=578963&r2=578964&view=diff
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Mon Sep 24 13:57:20 2007
@@ -93,6 +93,9 @@
HADOOP-1500. Fix typographical errors in the DFS WebUI.
(Nigel Daley via dhruba)
+ HADOOP-1076. Periodic checkpoint can continue even if an earlier
+ checkpoint encountered an error. (Dhruba Borthakur via dhruba)
+
HADOOP-1887. The Namenode encounters an ArrayIndexOutOfBoundsException
while listing a directory that had a file that was
being actively written to. (Dhruba Borthakur via dhruba)
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java?rev=578964&r1=578963&r2=578964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/ClientProtocol.java Mon Sep 24 13:57:20 2007
@@ -34,8 +34,9 @@
* 16 : removed deprecated obtainLock() and releaseLock().
* 17 : getBlockSize replaced by getPreferredBlockSize
* 18 : datanodereport returns dead, live or all nodes.
+ * 19 : rollEditLog() returns a token to uniquely identify the editfile.
*/
- public static final long versionID = 18L;
+ public static final long versionID = 19L;
///////////////////////////////////////
// File contents
@@ -326,11 +327,11 @@
/**
* Closes the current edit log and opens a new one. The
- * call fails if there are already two or more edits log files or
- * if the file system is in SafeMode.
+ * call fails if the file system is in SafeMode.
+ * Returns a unique token to identify this transaction.
* @throws IOException
*/
- public void rollEditLog() throws IOException;
+ public long rollEditLog() throws IOException;
/**
* Rolls the fsImage log. It removes the old fsImage, copies the
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java?rev=578964&r1=578963&r2=578964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSConstants.java Mon Sep 24 13:57:20 2007
@@ -133,7 +133,10 @@
// type of the datanode report
public static enum DatanodeReportType {ALL, LIVE, DEAD }
-
+
+ // checkpoint states
+ public enum CheckpointStates{ START, ROLLED_EDITS, UPLOAD_START, UPLOAD_DONE; }
+
/**
* Type of the node
*/
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSEditLog.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSEditLog.java?rev=578964&r1=578963&r2=578964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSEditLog.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSEditLog.java Mon Sep 24 13:57:20 2007
@@ -393,7 +393,6 @@
+ " for version " + logVersion);
DatanodeID nodeID = new DatanodeID();
nodeID.readFields(in);
- DatanodeDescriptor node = fsNamesys.getDatanode(nodeID);
//Datanodes are not persistent any more.
break;
}
@@ -577,13 +576,21 @@
/**
* Closes the current edit log and opens edits.new.
+ * Returns the lastModified time of the edits log.
*/
synchronized void rollEditLog() throws IOException {
//
- // If edits.new already exists, then return error.
+ // If edits.new already exists in some directory, verify it
+ // exists in all directories.
//
if (existsNew()) {
- throw new IOException("Attempt to roll edit log but edits.new exists");
+ for (int idx = 0; idx < getNumStorageDirs(); idx++) {
+ if (!getEditNewFile(idx).exists()) {
+ throw new IOException("Inconsistent existance of edits.new " +
+ getEditNewFile(idx));
+ }
+ }
+ return; // nothing to do, edits.new exists!
}
close(); // close existing edit log
@@ -644,5 +651,12 @@
*/
synchronized File getFsEditName() throws IOException {
return getEditFile(0);
+ }
+
+ /**
+ * Returns the timestamp of the edit log
+ */
+ synchronized long getFsEditTime() throws IOException {
+ return getEditFile(0).lastModified();
}
}
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java?rev=578964&r1=578963&r2=578964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java Mon Sep 24 13:57:20 2007
@@ -33,6 +33,7 @@
import java.io.PrintWriter;
import java.util.*;
import java.util.Map.Entry;
+import java.text.SimpleDateFormat;
/***************************************************
* FSNamesystem does the actual bookkeeping work for the
@@ -195,6 +196,13 @@
private HostsFileReader hostsReader;
private Daemon dnthread = null;
+ // can fs-image be rolled?
+ volatile private CheckpointStates ckptState = CheckpointStates.START;
+
+ private static final SimpleDateFormat DATE_FORM =
+ new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+
+
/**
* dirs is a list of directories where the filesystem directory state
* is stored
@@ -3447,13 +3455,15 @@
return getEditLog().getEditLogSize();
}
- synchronized void rollEditLog() throws IOException {
+ synchronized long rollEditLog() throws IOException {
if (isInSafeMode()) {
throw new SafeModeException("Checkpoint not created",
safeMode);
}
LOG.info("Roll Edit Log from " + Server.getRemoteAddress());
getEditLog().rollEditLog();
+ ckptState = CheckpointStates.ROLLED_EDITS;
+ return getEditLog().getFsEditTime();
}
synchronized void rollFSImage() throws IOException {
@@ -3462,13 +3472,46 @@
throw new SafeModeException("Checkpoint not created",
safeMode);
}
+ if (ckptState != CheckpointStates.UPLOAD_DONE) {
+ throw new IOException("Cannot roll fsImage before rolling edits log.");
+ }
dir.fsImage.rollFSImage();
+ ckptState = CheckpointStates.START;
}
File getFsEditName() throws IOException {
return getEditLog().getFsEditName();
}
-
+
+ /*
+ * This is called just before a new checkpoint is uploaded to the
+ * namenode.
+ */
+ synchronized void validateCheckpointUpload(long token) throws IOException {
+ if (ckptState != CheckpointStates.ROLLED_EDITS) {
+ throw new IOException("Namenode is not expecting an new image " +
+ ckptState);
+ }
+ // verify token
+ long modtime = getEditLog().getFsEditTime();
+ if (token != modtime) {
+ throw new IOException("Namenode has an edit log with timestamp of " +
+ DATE_FORM.format(new Date(modtime)) +
+ " but new checkpoint was created using editlog " +
+ " with timestamp " +
+ DATE_FORM.format(new Date(token)) +
+ ". Checkpoint Aborted.");
+ }
+ ckptState = CheckpointStates.UPLOAD_START;
+ }
+
+ /*
+ * This is called when a checkpoint upload finishes successfully.
+ */
+ synchronized void checkpointUploadDone() {
+ ckptState = CheckpointStates.UPLOAD_DONE;
+ }
+
/**
* Returns whether the given block is one pointed-to by a file.
*/
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/GetImageServlet.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/GetImageServlet.java?rev=578964&r1=578963&r2=578964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/GetImageServlet.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/GetImageServlet.java Mon Sep 24 13:57:20 2007
@@ -20,8 +20,6 @@
import java.util.*;
import java.io.*;
import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.mapred.StatusHttpServer;
import org.apache.commons.logging.*;
import javax.servlet.ServletContext;
import javax.servlet.ServletException;
@@ -58,8 +56,10 @@
nn.getFsEditName());
} else if (ff.putImage()) {
// issue a HTTP get request to download the new fsimage
+ nn.validateCheckpointUpload(ff.getToken());
TransferFsImage.getFileClient(ff.getInfoServer(), "getimage=1",
nn.getFsImageNameCheckpoint());
+ nn.checkpointUploadDone();
}
} catch (IOException ie) {
StringUtils.stringifyException(ie);
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java?rev=578964&r1=578963&r2=578964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java Mon Sep 24 13:57:20 2007
@@ -521,8 +521,8 @@
/**
* Roll the edit log.
*/
- public void rollEditLog() throws IOException {
- namesystem.rollEditLog();
+ public long rollEditLog() throws IOException {
+ return namesystem.rollEditLog();
}
/**
@@ -709,6 +709,20 @@
*/
public File[] getFsImageNameCheckpoint() throws IOException {
return getFSImage().getFsImageNameCheckpoint();
+ }
+
+ /**
+ * Validates that this is a valid checkpoint upload request
+ */
+ public void validateCheckpointUpload(long token) throws IOException {
+ namesystem.validateCheckpointUpload(token);
+ }
+
+ /**
+ * Indicates that a new checkpoint has been successfully uploaded.
+ */
+ public void checkpointUploadDone() {
+ namesystem.checkpointUploadDone();
}
/**
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/SecondaryNameNode.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/SecondaryNameNode.java?rev=578964&r1=578963&r2=578964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/SecondaryNameNode.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/SecondaryNameNode.java Mon Sep 24 13:57:20 2007
@@ -223,11 +223,12 @@
/**
* Copy the new fsimage into the NameNode
*/
- private void putFSImage() throws IOException {
+ private void putFSImage(long token) throws IOException {
String fsName = getInfoServer();
String fileid = "putimage=1&port=" + infoPort +
"&machine=" +
- InetAddress.getLocalHost().getHostAddress();
+ InetAddress.getLocalHost().getHostAddress() +
+ "&token=" + token;
LOG.info("Posted URL " + fsName + fileid);
TransferFsImage.getFileClient(fsName, fileid, (File[])null);
}
@@ -257,8 +258,9 @@
//
// Tell the namenode to start logging transactions in a new edit file
+ // Retuns a token that would be used to upload the merged image.
//
- namenode.rollEditLog();
+ long token = namenode.rollEditLog();
//
// error simulation code for junit test
@@ -276,7 +278,7 @@
// Upload the new image into the NameNode. Then tell the Namenode
// to make this new uploaded image as the most current image.
//
- putFSImage();
+ putFSImage(token);
//
// error simulation code for junit test
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/TransferFsImage.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/TransferFsImage.java?rev=578964&r1=578963&r2=578964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/TransferFsImage.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/TransferFsImage.java Mon Sep 24 13:57:20 2007
@@ -34,6 +34,7 @@
private boolean isPutImage;
private int remoteport;
private String machineName;
+ private long token;
/**
* File downloader.
@@ -50,6 +51,7 @@
isGetImage = isGetEdit = isPutImage = false;
remoteport = 0;
machineName = null;
+ token = 0;
for (Iterator<String> it = pmap.keySet().iterator(); it.hasNext();) {
String key = it.next();
@@ -63,6 +65,8 @@
remoteport = new Integer(pmap.get("port")[0]).intValue();
} else if (key.equals("machine")) {
machineName = pmap.get("machine")[0];
+ } else if (key.equals("token")) {
+ token = new Long(pmap.get("token")[0]).longValue();
}
}
if ((isGetImage && isGetEdit) ||
@@ -81,6 +85,10 @@
boolean putImage() {
return isPutImage;
+ }
+
+ long getToken() {
+ return token;
}
String getInfoServer() throws IOException{
Modified: lucene/hadoop/trunk/src/test/org/apache/hadoop/dfs/TestCheckpoint.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/org/apache/hadoop/dfs/TestCheckpoint.java?rev=578964&r1=578963&r2=578964&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/test/org/apache/hadoop/dfs/TestCheckpoint.java (original)
+++ lucene/hadoop/trunk/src/test/org/apache/hadoop/dfs/TestCheckpoint.java Mon Sep 24 13:57:20 2007
@@ -190,8 +190,7 @@
try {
assertTrue(!fileSys.exists(file1));
//
- // Make the checkpoint fail after rolling the
- // edit log.
+ // Make the checkpoint fail after uploading the new fsimage.
//
SecondaryNameNode secondary = new SecondaryNameNode(conf);
secondary.initializeErrorSimulationEvent(2);
@@ -235,6 +234,72 @@
}
}
+ /*
+ * Simulate a secondary namenode crash after rolling the edit log.
+ */
+ private void testSecondaryNamenodeError3(Configuration conf)
+ throws IOException {
+ System.out.println("Starting testSecondaryNamenodeError 31");
+ Path file1 = new Path("checkpointzz.dat");
+ MiniDFSCluster cluster = new MiniDFSCluster(conf, numDatanodes,
+ false, null);
+ cluster.waitActive();
+ FileSystem fileSys = cluster.getFileSystem();
+ try {
+ assertTrue(!fileSys.exists(file1));
+ //
+ // Make the checkpoint fail after rolling the edit log.
+ //
+ SecondaryNameNode secondary = new SecondaryNameNode(conf);
+ secondary.initializeErrorSimulationEvent(2);
+ secondary.setErrorSimulation(0);
+
+ try {
+ secondary.doCheckpoint(); // this should fail
+ assertTrue(false);
+ } catch (IOException e) {
+ }
+ secondary.shutdown(); // secondary namenode crash!
+
+ // start new instance of secondary and verify that
+ // a new rollEditLog suceedes inspite of the fact that
+ // edits.new already exists.
+ //
+ secondary = new SecondaryNameNode(conf);
+ secondary.doCheckpoint(); // this should work correctly
+ secondary.shutdown();
+
+ //
+ // Create a new file
+ //
+ writeFile(fileSys, file1, replication);
+ checkFile(fileSys, file1, replication);
+ } finally {
+ fileSys.close();
+ cluster.shutdown();
+ }
+
+ //
+ // Restart cluster and verify that file exists.
+ // Then take another checkpoint to verify that the
+ // namenode restart accounted for the twice-rolled edit logs.
+ //
+ System.out.println("Starting testSecondaryNamenodeError 32");
+ cluster = new MiniDFSCluster(conf, numDatanodes, false, null);
+ cluster.waitActive();
+ fileSys = cluster.getFileSystem();
+ try {
+ checkFile(fileSys, file1, replication);
+ cleanupFile(fileSys, file1);
+ SecondaryNameNode secondary = new SecondaryNameNode(conf);
+ secondary.doCheckpoint();
+ secondary.shutdown();
+ } finally {
+ fileSys.close();
+ cluster.shutdown();
+ }
+ }
+
/**
* Tests checkpoint in DFS.
*/
@@ -322,6 +387,7 @@
testSecondaryNamenodeError1(conf);
testSecondaryNamenodeError2(conf);
+ testSecondaryNamenodeError3(conf);
testNamedirError(conf, namedirs);
}
}