Browse Source

HADOOP-3724. Fixes two problems related to storing and recovering lease
in the fsimage. (dhruba)



git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@679866 13f79535-47bb-0310-9956-ffa450edef68

Dhruba Borthakur 17 năm trước cách đây
mục cha
commit
056fa2fb00

+ 3 - 0
CHANGES.txt

@@ -941,6 +941,9 @@ Release 0.18.0 - Unreleased
     HADOOP-3820. Fixes two problems in the gridmix-env - a syntax error, and a 
     wrong definition of USE_REAL_DATASET by default. (Arun Murthy via ddas)
 
+    HADOOP-3724. Fixes two problems related to storing and recovering lease
+    in the fsimage. (dhruba)
+
 Release 0.17.2 - Unreleased
 
   BUG FIXES

+ 8 - 1
src/hdfs/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java

@@ -385,7 +385,14 @@ public class DatanodeDescriptor extends DatanodeInfo {
         continue;
       }
       if(storedBlock.findDatanode(this) < 0) {// Known block, but not on the DN
-        toAdd.add(storedBlock);
+        // if the size differs from what is in the blockmap, then return
+        // the new block. addStoredBlock will then pick up the right size of this
+        // block and will update the block object in the BlocksMap
+        if (storedBlock.getNumBytes() != iblk.getNumBytes()) {
+          toAdd.add(new Block(iblk));
+        } else {
+          toAdd.add(storedBlock);
+        }
         continue;
       }
       // move block to the head of the list

+ 4 - 1
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -635,8 +635,11 @@ class FSDirectory implements FSConstants, Closeable {
                               "failed to remove " + path);
       } 
       rootDir.addNode(path, newnode); 
+      int index = 0;
       for (Block b : newnode.getBlocks()) {
-        namesystem.blocksMap.addINode(b, newnode);
+        BlockInfo info = namesystem.blocksMap.addINode(b, newnode);
+        newnode.setBlock(index, info); // inode refers to the block in BlocksMap
+        index++;
       }
     }
   }

+ 3 - 1
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.fs.permission.*;
+import org.apache.hadoop.hdfs.protocol.DFSFileInfo;
 
 /**
  * FSEditLog maintains a log of the namespace modifications.
@@ -562,8 +563,9 @@ public class FSEditLog {
             String s = FSImage.readString(in);
             String d = FSImage.readString(in);
             timestamp = readLong(in);
+            DFSFileInfo dinfo = fsDir.getFileInfo(d);
             fsDir.unprotectedRenameTo(s, d, timestamp);
-            fsNamesys.changeLease(s, d);
+            fsNamesys.changeLease(s, d, dinfo);
             break;
           }
           case OP_DELETE: {

+ 10 - 4
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -1459,8 +1459,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
       checkAncestorAccess(actualdst, FsAction.WRITE);
     }
 
+    DFSFileInfo dinfo = dir.getFileInfo(dst);
     if (dir.renameTo(src, dst)) {
-      changeLease(src, dst);     // update lease with new filename
+      changeLease(src, dst, dinfo);     // update lease with new filename
       return true;
     }
     return false;
@@ -4366,12 +4367,17 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean {
   // rename was successful. If any part of the renamed subtree had
   // files that were being written to, update with new filename.
   //
-  void changeLease(String src, String dst) throws IOException {
+  void changeLease(String src, String dst, DFSFileInfo dinfo) 
+                   throws IOException {
     String overwrite;
     String replaceBy;
 
-    DFSFileInfo dinfo = dir.getFileInfo(dst);
-    if (dinfo.isDir()) {
+    boolean destinationExisted = true;
+    if (dinfo == null) {
+      destinationExisted = false;
+    }
+
+    if (destinationExisted && dinfo.isDir()) {
       Path spath = new Path(src);
       overwrite = spath.getParent().toString() + Path.SEPARATOR;
       replaceBy = dst + Path.SEPARATOR;

+ 26 - 2
src/test/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -75,7 +75,7 @@ public class TestFileCreation extends junit.framework.TestCase {
   //
   // writes to file but does not close it
   //
-  private void writeFile(FSDataOutputStream stm) throws IOException {
+  static void writeFile(FSDataOutputStream stm) throws IOException {
     byte[] buffer = new byte[fileSize];
     Random rand = new Random(seed);
     rand.nextBytes(buffer);
@@ -131,10 +131,11 @@ public class TestFileCreation extends junit.framework.TestCase {
     // do a sanity check. Read the file
     byte[] actual = new byte[numBlocks * blockSize];
     stm.readFully(0, actual);
+    stm.close();
     checkData(actual, 0, expected, "Read 1");
   }
 
-  private void checkData(byte[] actual, int from, byte[] expected, String message) {
+  static private void checkData(byte[] actual, int from, byte[] expected, String message) {
     for (int idx = 0; idx < actual.length; idx++) {
       assertEquals(message+" byte "+(from+idx)+" differs. expected "+
                    expected[from+idx]+" actual "+actual[idx],
@@ -143,6 +144,29 @@ public class TestFileCreation extends junit.framework.TestCase {
     }
   }
 
+  static void checkFullFile(FileSystem fs, Path name) throws IOException {
+    FileStatus stat = fs.getFileStatus(name);
+    BlockLocation[] locations = fs.getFileBlockLocations(stat, 0, 
+                                                         fileSize);
+    for (int idx = 0; idx < locations.length; idx++) {
+      String[] hosts = locations[idx].getNames();
+      for (int i = 0; i < hosts.length; i++) {
+        System.out.print( hosts[i] + " ");
+      }
+      System.out.println(" off " + locations[idx].getOffset() +
+                         " len " + locations[idx].getLength());
+    }
+
+    byte[] expected = new byte[fileSize];
+    Random rand = new Random(seed);
+    rand.nextBytes(expected);
+    FSDataInputStream stm = fs.open(name);
+    byte[] actual = new byte[fileSize];
+    stm.readFully(0, actual);
+    checkData(actual, 0, expected, "Read 2");
+    stm.close();
+  }
+
   /**
    * Test that file data becomes available before file is closed.
    */

+ 306 - 0
src/test/org/apache/hadoop/hdfs/TestRenameWhileOpen.java

@@ -0,0 +1,306 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.log4j.Level;
+
+public class TestRenameWhileOpen extends junit.framework.TestCase {
+  {
+    ((Log4JLogger)NameNode.stateChangeLog).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger)FSNamesystem.LOG).getLogger().setLevel(Level.ALL);
+  }
+
+  /**
+   * open /user/dir1/file1 /user/dir2/file2
+   * mkdir /user/dir3
+   * move /user/dir1 /user/dir3
+   */
+  public void testWhileOpenRenameParent() throws IOException {
+    Configuration conf = new Configuration();
+    final int MAX_IDLE_TIME = 2000; // 2s
+    conf.setInt("ipc.client.connection.maxidletime", MAX_IDLE_TIME);
+    conf.setInt("heartbeat.recheck.interval", 1000);
+    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt("dfs.safemode.threshold.pct", 1);
+
+    // create cluster
+    System.out.println("Test 1*****************************");
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    FileSystem fs = null;
+    try {
+      cluster.waitActive();
+      fs = cluster.getFileSystem();
+      final int nnport = cluster.getNameNodePort();
+
+      // create file1.
+      Path dir1 = new Path("/user/dir1");
+      Path file1 = new Path(dir1, "file1");
+      FSDataOutputStream stm1 = TestFileCreation.createFile(fs, file1, 1);
+      System.out.println("testFileCreationDeleteParent: "
+          + "Created file " + file1);
+      TestFileCreation.writeFile(stm1);
+      stm1.sync();
+
+      // create file2.
+      Path dir2 = new Path("/user/dir2");
+      Path file2 = new Path(dir2, "file2");
+      FSDataOutputStream stm2 = TestFileCreation.createFile(fs, file2, 1);
+      System.out.println("testFileCreationDeleteParent: "
+          + "Created file " + file2);
+      TestFileCreation.writeFile(stm2);
+      stm2.sync();
+
+      // move dir1 while file1 is open
+      Path dir3 = new Path("/user/dir3");
+      fs.mkdirs(dir3);
+      fs.rename(dir1, dir3);
+
+      // restart cluster with the same namenode port as before.
+      // This ensures that leases are persisted in fsimage.
+      cluster.shutdown();
+      try {Thread.sleep(2*MAX_IDLE_TIME);} catch (InterruptedException e) {}
+      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
+                                   null, null, null);
+      cluster.waitActive();
+
+      // restart cluster yet again. This triggers the code to read in
+      // persistent leases from fsimage.
+      cluster.shutdown();
+      try {Thread.sleep(5000);} catch (InterruptedException e) {}
+      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
+                                   null, null, null);
+      cluster.waitActive();
+      fs = cluster.getFileSystem();
+
+      Path newfile = new Path("/user/dir3/dir1", "file1");
+      assertTrue(!fs.exists(file1));
+      assertTrue(fs.exists(file2));
+      assertTrue(fs.exists(newfile));
+      TestFileCreation.checkFullFile(fs, newfile);
+    } finally {
+      fs.close();
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * open /user/dir1/file1 /user/dir2/file2
+   * move /user/dir1 /user/dir3
+   */
+  public void testWhileOpenRenameParentToNonexistentDir() throws IOException {
+    Configuration conf = new Configuration();
+    final int MAX_IDLE_TIME = 2000; // 2s
+    conf.setInt("ipc.client.connection.maxidletime", MAX_IDLE_TIME);
+    conf.setInt("heartbeat.recheck.interval", 1000);
+    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt("dfs.safemode.threshold.pct", 1);
+    System.out.println("Test 2************************************");
+
+    // create cluster
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    FileSystem fs = null;
+    try {
+      cluster.waitActive();
+      fs = cluster.getFileSystem();
+      final int nnport = cluster.getNameNodePort();
+
+      // create file1.
+      Path dir1 = new Path("/user/dir1");
+      Path file1 = new Path(dir1, "file1");
+      FSDataOutputStream stm1 = TestFileCreation.createFile(fs, file1, 1);
+      System.out.println("testFileCreationDeleteParent: "
+          + "Created file " + file1);
+      TestFileCreation.writeFile(stm1);
+      stm1.sync();
+
+      // create file2.
+      Path dir2 = new Path("/user/dir2");
+      Path file2 = new Path(dir2, "file2");
+      FSDataOutputStream stm2 = TestFileCreation.createFile(fs, file2, 1);
+      System.out.println("testFileCreationDeleteParent: "
+          + "Created file " + file2);
+      TestFileCreation.writeFile(stm2);
+      stm2.sync();
+
+      // move dir1 while file1 is open
+      Path dir3 = new Path("/user/dir3");
+      fs.rename(dir1, dir3);
+
+      // restart cluster with the same namenode port as before.
+      // This ensures that leases are persisted in fsimage.
+      cluster.shutdown();
+      try {Thread.sleep(2*MAX_IDLE_TIME);} catch (InterruptedException e) {}
+      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
+                                   null, null, null);
+      cluster.waitActive();
+
+      // restart cluster yet again. This triggers the code to read in
+      // persistent leases from fsimage.
+      cluster.shutdown();
+      try {Thread.sleep(5000);} catch (InterruptedException e) {}
+      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
+                                   null, null, null);
+      cluster.waitActive();
+      fs = cluster.getFileSystem();
+
+      Path newfile = new Path("/user/dir3", "file1");
+      assertTrue(!fs.exists(file1));
+      assertTrue(fs.exists(file2));
+      assertTrue(fs.exists(newfile));
+      TestFileCreation.checkFullFile(fs, newfile);
+    } finally {
+      fs.close();
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * open /user/dir1/file1 
+   * mkdir /user/dir2
+   * move /user/dir1/file1 /user/dir2/
+   */
+  public void testWhileOpenRenameToExistentDirectory() throws IOException {
+    Configuration conf = new Configuration();
+    final int MAX_IDLE_TIME = 2000; // 2s
+    conf.setInt("ipc.client.connection.maxidletime", MAX_IDLE_TIME);
+    conf.setInt("heartbeat.recheck.interval", 1000);
+    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt("dfs.safemode.threshold.pct", 1);
+    System.out.println("Test 3************************************");
+
+    // create cluster
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    FileSystem fs = null;
+    try {
+      cluster.waitActive();
+      fs = cluster.getFileSystem();
+      final int nnport = cluster.getNameNodePort();
+
+      // create file1.
+      Path dir1 = new Path("/user/dir1");
+      Path file1 = new Path(dir1, "file1");
+      FSDataOutputStream stm1 = TestFileCreation.createFile(fs, file1, 1);
+      System.out.println("testFileCreationDeleteParent: " +
+                         "Created file " + file1);
+      TestFileCreation.writeFile(stm1);
+      stm1.sync();
+
+      Path dir2 = new Path("/user/dir2");
+      fs.mkdirs(dir2);
+
+      fs.rename(file1, dir2);
+
+      // restart cluster with the same namenode port as before.
+      // This ensures that leases are persisted in fsimage.
+      cluster.shutdown();
+      try {Thread.sleep(2*MAX_IDLE_TIME);} catch (InterruptedException e) {}
+      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
+                                   null, null, null);
+      cluster.waitActive();
+
+      // restart cluster yet again. This triggers the code to read in
+      // persistent leases from fsimage.
+      cluster.shutdown();
+      try {Thread.sleep(5000);} catch (InterruptedException e) {}
+      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
+                                   null, null, null);
+      cluster.waitActive();
+      fs = cluster.getFileSystem();
+
+      Path newfile = new Path("/user/dir2", "file1");
+      assertTrue(!fs.exists(file1));
+      assertTrue(fs.exists(newfile));
+      TestFileCreation.checkFullFile(fs, newfile);
+    } finally {
+      fs.close();
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * open /user/dir1/file1 
+   * move /user/dir1/file1 /user/dir2/
+   */
+  public void testWhileOpenRenameToNonExistentDirectory() throws IOException {
+    Configuration conf = new Configuration();
+    final int MAX_IDLE_TIME = 2000; // 2s
+    conf.setInt("ipc.client.connection.maxidletime", MAX_IDLE_TIME);
+    conf.setInt("heartbeat.recheck.interval", 1000);
+    conf.setInt("dfs.heartbeat.interval", 1);
+    conf.setInt("dfs.safemode.threshold.pct", 1);
+    System.out.println("Test 4************************************");
+
+    // create cluster
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    FileSystem fs = null;
+    try {
+      cluster.waitActive();
+      fs = cluster.getFileSystem();
+      final int nnport = cluster.getNameNodePort();
+
+      // create file1.
+      Path dir1 = new Path("/user/dir1");
+      Path file1 = new Path(dir1, "file1");
+      FSDataOutputStream stm1 = TestFileCreation.createFile(fs, file1, 1);
+      System.out.println("testFileCreationDeleteParent: "
+          + "Created file " + file1);
+      TestFileCreation.writeFile(stm1);
+      stm1.sync();
+
+      Path dir2 = new Path("/user/dir2");
+
+      fs.rename(file1, dir2);
+
+      // restart cluster with the same namenode port as before.
+      // This ensures that leases are persisted in fsimage.
+      cluster.shutdown();
+      try {Thread.sleep(2*MAX_IDLE_TIME);} catch (InterruptedException e) {}
+      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
+                                   null, null, null);
+      cluster.waitActive();
+
+      // restart cluster yet again. This triggers the code to read in
+      // persistent leases from fsimage.
+      cluster.shutdown();
+      try {Thread.sleep(5000);} catch (InterruptedException e) {}
+      cluster = new MiniDFSCluster(nnport, conf, 1, false, true, 
+                                   null, null, null);
+      cluster.waitActive();
+      fs = cluster.getFileSystem();
+
+      Path newfile = new Path("/user", "dir2");
+      assertTrue(!fs.exists(file1));
+      assertTrue(fs.exists(newfile));
+      TestFileCreation.checkFullFile(fs, newfile);
+    } finally {
+      fs.close();
+      cluster.shutdown();
+    }
+  }
+}