Переглянути джерело

HDFS-4927. Merging change r1496351 from branch-2 to branch-2.1-beta.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.1-beta@1496352 13f79535-47bb-0310-9956-ffa450edef68
Chris Nauroth 12 роки тому
батько
коміт
5148535404

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -381,6 +381,9 @@ Release 2.1.0-beta - UNRELEASED
 
     HDFS-4205. fsck fails with symlinks. (jlowe)
 
+    HDFS-4927. CreateEditsLog creates inodes with an invalid inode ID, which then
+    cannot be loaded by a namenode. (cnauroth)
+
   BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
 
     HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.

+ 8 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.namenode.INodeId;
 
 /**
  * 
@@ -62,8 +63,9 @@ public class CreateEditsLog {
     
     PermissionStatus p = new PermissionStatus("joeDoe", "people",
                                       new FsPermission((short)0777));
-    INodeDirectory dirInode = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID,
-        null, p, 0L);
+    INodeId inodeId = new INodeId();
+    INodeDirectory dirInode = new INodeDirectory(inodeId.nextValue(), null, p,
+      0L);
     editLog.logMkDir(BASE_PATH, dirInode);
     long blockSize = 10;
     BlockInfo[] blocks = new BlockInfo[blocksPerFile];
@@ -82,8 +84,8 @@ public class CreateEditsLog {
       }
 
       INodeFileUnderConstruction inode = new INodeFileUnderConstruction(
-          INodeId.GRANDFATHER_INODE_ID, null, replication, 0, blockSize,
-          blocks, p, "", "", null);
+          inodeId.nextValue(), null, replication, 0, blockSize, blocks, p, "",
+          "", null);
      // Append path to filename with information about blockIDs 
       String path = "_" + iF + "_B" + blocks[0].getBlockId() + 
                     "_to_B" + blocks[blocksPerFile-1].getBlockId() + "_";
@@ -92,12 +94,11 @@ public class CreateEditsLog {
       // Log the new sub directory in edits
       if ((iF % nameGenerator.getFilesPerDirectory())  == 0) {
         String currentDir = nameGenerator.getCurrentDir();
-        dirInode = new INodeDirectory(INodeId.GRANDFATHER_INODE_ID, null, p, 0L);
+        dirInode = new INodeDirectory(inodeId.nextValue(), null, p, 0L);
         editLog.logMkDir(currentDir, dirInode);
       }
       editLog.logOpenFile(filePath, new INodeFileUnderConstruction(
-          INodeId.GRANDFATHER_INODE_ID, p, replication, 0, blockSize, "", "",
-          null));
+          inodeId.nextValue(), p, replication, 0, blockSize, "", "", null));
       editLog.logCloseFile(filePath, inode);
 
       if (currentBlockId - bidAtSync >= 2000) { // sync every 2K blocks

+ 112 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCreateEditsLog.java

@@ -0,0 +1,112 @@
+/**
+ * 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.server.namenode;
+
+import static org.junit.Assert.fail;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
+
+import java.io.File;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.common.Util;
+
+/**
+ * Tests the CreateEditsLog utility.
+ */
+public class TestCreateEditsLog {
+
+  private static final File HDFS_DIR = new File(
+    MiniDFSCluster.getBaseDirectory()).getAbsoluteFile();
+  private static final File TEST_DIR = new File(
+    System.getProperty("test.build.data", "build/test/data"),
+    "TestCreateEditsLog").getAbsoluteFile();
+
+  private MiniDFSCluster cluster;
+
+  @Before
+  public void setUp() throws Exception {
+    deleteIfExists(HDFS_DIR);
+    deleteIfExists(TEST_DIR);
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }
+    deleteIfExists(HDFS_DIR);
+    deleteIfExists(TEST_DIR);
+  }
+
+  /**
+   * Tests that an edits log created using CreateEditsLog is valid and can be
+   * loaded successfully by a namenode.
+   */
+  @Test(timeout=60000)
+  public void testCanLoadCreatedEditsLog() throws Exception {
+    // Format namenode.
+    HdfsConfiguration conf = new HdfsConfiguration();
+    File nameDir = new File(HDFS_DIR, "name");
+    conf.set(DFS_NAMENODE_NAME_DIR_KEY, Util.fileAsURI(nameDir).toString());
+    DFSTestUtil.formatNameNode(conf);
+
+    // Call CreateEditsLog and move the resulting edits to the name dir.
+    CreateEditsLog.main(new String[] { "-f", "1000", "0", "1", "-d",
+      TEST_DIR.getAbsolutePath() });
+    Path editsWildcard = new Path(TEST_DIR.getAbsolutePath(), "*");
+    FileContext localFc = FileContext.getLocalFSFileContext();
+    for (FileStatus edits: localFc.util().globStatus(editsWildcard)) {
+      Path src = edits.getPath();
+      Path dst = new Path(new File(nameDir, "current").getAbsolutePath(),
+        src.getName());
+      localFc.rename(src, dst);
+    }
+
+    // Start a namenode to try to load the edits.
+    cluster = new MiniDFSCluster.Builder(conf)
+      .format(false)
+      .manageNameDfsDirs(false)
+      .waitSafeMode(false)
+      .build();
+    cluster.waitClusterUp();
+
+    // Test successful, because no exception thrown.
+  }
+
+  /**
+   * Fully delete the given directory if it exists.
+   * 
+   * @param file File to delete
+   */
+  private static void deleteIfExists(File file) {
+    if (file.exists() && !FileUtil.fullyDelete(file)) {
+      fail("Could not delete  '" + file + "'");
+    }
+  }
+}