Author: dhruba
Date: Wed Oct 3 16:18:55 2007
New Revision: 581746
URL: http://svn.apache.org/viewvc?rev=581746&view=rev
Log:
HADOOP-1978. Name-node removes edits.new after a successful startup.
(Konstantin Shvachko via dhruba)
svn merge -c 581745 from trunk into branch 0.14.
Modified:
lucene/hadoop/branches/branch-0.14/CHANGES.txt
lucene/hadoop/branches/branch-0.14/src/java/org/apache/hadoop/dfs/FSEditLog.java
lucene/hadoop/branches/branch-0.14/src/java/org/apache/hadoop/dfs/FSImage.java
lucene/hadoop/branches/branch-0.14/src/test/org/apache/hadoop/dfs/TestCheckpoint.java
lucene/hadoop/branches/branch-0.14/src/test/org/apache/hadoop/dfs/TestFsck.java
Modified: lucene/hadoop/branches/branch-0.14/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/branches/branch-0.14/CHANGES.txt?rev=581746&r1=581745&r2=581746&view=diff
==============================================================================
--- lucene/hadoop/branches/branch-0.14/CHANGES.txt (original)
+++ lucene/hadoop/branches/branch-0.14/CHANGES.txt Wed Oct 3 16:18:55 2007
@@ -23,6 +23,9 @@
HADOOP-1970. Fix deadlock in progress reporting in the task. (Vivek
Ratan via omalley)
+ HADOOP-1978. Name-node removes edits.new after a successful startup.
+ (Konstantin Shvachko via dhruba)
+
Release 0.14.1 - 2007-09-04
BUG FIXES
Modified: lucene/hadoop/branches/branch-0.14/src/java/org/apache/hadoop/dfs/FSEditLog.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/branches/branch-0.14/src/java/org/apache/hadoop/dfs/FSEditLog.java?rev=581746&r1=581745&r2=581746&view=diff
==============================================================================
--- lucene/hadoop/branches/branch-0.14/src/java/org/apache/hadoop/dfs/FSEditLog.java (original)
+++ lucene/hadoop/branches/branch-0.14/src/java/org/apache/hadoop/dfs/FSEditLog.java Wed Oct
3 16:18:55 2007
@@ -22,7 +22,6 @@
import java.io.DataOutputStream;
import java.io.EOFException;
import java.io.File;
-import java.io.FileDescriptor;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
@@ -52,19 +51,16 @@
private long lastSyncTime;
static class EditLogOutputStream extends DataOutputStream {
- private FileDescriptor fd;
-
EditLogOutputStream(File name) throws IOException {
super(new FileOutputStream(name, true)); // open for append
- this.fd = ((FileOutputStream)out).getFD();
}
void flushAndSync() throws IOException {
- this.flush();
- this.fd.sync();
+ ((FileOutputStream)out).getChannel().force(true);
}
void create() throws IOException {
+ ((FileOutputStream)out).getChannel().truncate(0);
writeInt(FSConstants.LAYOUT_VERSION);
flushAndSync();
}
@@ -122,7 +118,6 @@
synchronized void createEditLogFile(File name) throws IOException {
EditLogOutputStream eStream = new EditLogOutputStream(name);
eStream.create();
- eStream.flushAndSync();
eStream.close();
}
Modified: lucene/hadoop/branches/branch-0.14/src/java/org/apache/hadoop/dfs/FSImage.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/branches/branch-0.14/src/java/org/apache/hadoop/dfs/FSImage.java?rev=581746&r1=581745&r2=581746&view=diff
==============================================================================
--- lucene/hadoop/branches/branch-0.14/src/java/org/apache/hadoop/dfs/FSImage.java (original)
+++ lucene/hadoop/branches/branch-0.14/src/java/org/apache/hadoop/dfs/FSImage.java Wed Oct
3 16:18:55 2007
@@ -743,6 +743,7 @@
/**
* Save the contents of the FS image
+ * and create empty edits.
*/
void saveFSImage() throws IOException {
editLog.createNewIfMissing();
@@ -750,6 +751,9 @@
StorageDirectory sd = getStorageDir(idx);
saveFSImage(getImageFile(sd, NameNodeFile.IMAGE_NEW));
editLog.createEditLogFile(getImageFile(sd, NameNodeFile.EDITS));
+ File editsNew = getImageFile(sd, NameNodeFile.EDITS_NEW);
+ if (editsNew.exists())
+ editLog.createEditLogFile(editsNew);
}
rollFSImage();
}
Modified: lucene/hadoop/branches/branch-0.14/src/test/org/apache/hadoop/dfs/TestCheckpoint.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/branches/branch-0.14/src/test/org/apache/hadoop/dfs/TestCheckpoint.java?rev=581746&r1=581745&r2=581746&view=diff
==============================================================================
--- lucene/hadoop/branches/branch-0.14/src/test/org/apache/hadoop/dfs/TestCheckpoint.java
(original)
+++ lucene/hadoop/branches/branch-0.14/src/test/org/apache/hadoop/dfs/TestCheckpoint.java
Wed Oct 3 16:18:55 2007
@@ -22,6 +22,7 @@
import java.util.Collection;
import java.util.Random;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.dfs.FSImage.NameNodeFile;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -163,6 +164,19 @@
System.out.println("Starting testSecondaryNamenodeError 2");
cluster = new MiniDFSCluster(conf, numDatanodes, false, null);
cluster.waitActive();
+ // Also check that the edits file is empty here
+ // and that temporary checkpoint files are gone.
+ FSImage image = cluster.getNameNode().getFSImage();
+ int nrDirs = image.getNumStorageDirs();
+ for(int idx = 0; idx < nrDirs; idx++) {
+ assertFalse(image.getImageFile(idx, NameNodeFile.IMAGE_NEW).exists());
+ assertFalse(image.getEditNewFile(idx).exists());
+ File edits = image.getEditFile(idx);
+ assertTrue(edits.exists()); // edits should exist and be empty
+ assertTrue(
+ (new RandomAccessFile(edits, "r")).length() == Integer.SIZE/Byte.SIZE);
+ }
+
fileSys = cluster.getFileSystem();
try {
checkFile(fileSys, file1, replication);
Modified: lucene/hadoop/branches/branch-0.14/src/test/org/apache/hadoop/dfs/TestFsck.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/branches/branch-0.14/src/test/org/apache/hadoop/dfs/TestFsck.java?rev=581746&r1=581745&r2=581746&view=diff
==============================================================================
--- lucene/hadoop/branches/branch-0.14/src/test/org/apache/hadoop/dfs/TestFsck.java (original)
+++ lucene/hadoop/branches/branch-0.14/src/test/org/apache/hadoop/dfs/TestFsck.java Wed Oct
3 16:18:55 2007
@@ -76,6 +76,7 @@
// bring up data nodes & cleanup cluster
cluster.startDataNodes(conf, 4, true, null, null);
+ cluster.waitActive();
util.cleanup(cluster.getFileSystem(), "/srcdat");
} finally {
if (cluster != null) { cluster.shutdown(); }
|