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 ji...@apache.org on 2015/05/01 09:01:52 UTC

[35/50] [abbrv] hadoop git commit: HDFS-8269. getBlockLocations() does not resolve the .reserved path and generates incorrect edit logs when updating the atime. Contributed by Haohui Mai.

HDFS-8269. getBlockLocations() does not resolve the .reserved path and generates incorrect edit logs when updating the atime. Contributed by Haohui Mai.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3dd6395b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3dd6395b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3dd6395b

Branch: refs/heads/HDFS-7240
Commit: 3dd6395bb2448e5b178a51c864e3c9a3d12e8bc9
Parents: 7947e5b
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Apr 29 11:12:45 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Wed Apr 29 11:12:45 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/namenode/FSNamesystem.java      |  63 ++++++---
 .../hdfs/server/namenode/NamenodeFsck.java      |   4 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |   8 +-
 .../server/namenode/TestGetBlockLocations.java  | 133 +++++++++++++++++++
 5 files changed, 188 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3dd6395b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 2dde356..a3f219b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -635,6 +635,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8273. FSNamesystem#Delete() should not call logSync() when holding the
     lock. (wheat9)
 
+    HDFS-8269. getBlockLocations() does not resolve the .reserved path and
+    generates incorrect edit logs when updating the atime. (wheat9)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3dd6395b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 0ec81d8..b938263 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -1697,13 +1697,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   static class GetBlockLocationsResult {
-    final INodesInPath iip;
+    final boolean updateAccessTime;
     final LocatedBlocks blocks;
     boolean updateAccessTime() {
-      return iip != null;
+      return updateAccessTime;
     }
-    private GetBlockLocationsResult(INodesInPath iip, LocatedBlocks blocks) {
-      this.iip = iip;
+    private GetBlockLocationsResult(
+        boolean updateAccessTime, LocatedBlocks blocks) {
+      this.updateAccessTime = updateAccessTime;
       this.blocks = blocks;
     }
   }
@@ -1712,34 +1713,58 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * Get block locations within the specified range.
    * @see ClientProtocol#getBlockLocations(String, long, long)
    */
-  LocatedBlocks getBlockLocations(String clientMachine, String src,
+  LocatedBlocks getBlockLocations(String clientMachine, String srcArg,
       long offset, long length) throws IOException {
     checkOperation(OperationCategory.READ);
     GetBlockLocationsResult res = null;
+    FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      res = getBlockLocations(src, offset, length, true, true);
+      res = getBlockLocations(pc, srcArg, offset, length, true, true);
     } catch (AccessControlException e) {
-      logAuditEvent(false, "open", src);
+      logAuditEvent(false, "open", srcArg);
       throw e;
     } finally {
       readUnlock();
     }
 
-    logAuditEvent(true, "open", src);
+    logAuditEvent(true, "open", srcArg);
 
     if (res.updateAccessTime()) {
+      byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(
+          srcArg);
+      String src = srcArg;
       writeLock();
       final long now = now();
       try {
         checkOperation(OperationCategory.WRITE);
-        INode inode = res.iip.getLastINode();
-        boolean updateAccessTime = now > inode.getAccessTime() +
-            getAccessTimePrecision();
+        /**
+         * Resolve the path again and update the atime only when the file
+         * exists.
+         *
+         * XXX: Races can still occur even after resolving the path again.
+         * For example:
+         *
+         * <ul>
+         *   <li>Get the block location for "/a/b"</li>
+         *   <li>Rename "/a/b" to "/c/b"</li>
+         *   <li>The second resolution still points to "/a/b", which is
+         *   wrong.</li>
+         * </ul>
+         *
+         * The behavior is incorrect but consistent with the one before
+         * HDFS-7463. A better fix is to change the edit log of SetTime to
+         * use inode id instead of a path.
+         */
+        src = dir.resolvePath(pc, srcArg, pathComponents);
+        final INodesInPath iip = dir.getINodesInPath(src, true);
+        INode inode = iip.getLastINode();
+        boolean updateAccessTime = inode != null &&
+            now > inode.getAccessTime() + getAccessTimePrecision();
         if (!isInSafeMode() && updateAccessTime) {
           boolean changed = FSDirAttrOp.setTimes(dir,
-              inode, -1, now, false, res.iip.getLatestSnapshotId());
+              inode, -1, now, false, iip.getLatestSnapshotId());
           if (changed) {
             getEditLog().logTimes(src, -1, now);
           }
@@ -1773,8 +1798,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @throws IOException
    */
   GetBlockLocationsResult getBlockLocations(
-      String src, long offset, long length, boolean needBlockToken,
-      boolean checkSafeMode) throws IOException {
+      FSPermissionChecker pc, String src, long offset, long length,
+      boolean needBlockToken, boolean checkSafeMode) throws IOException {
     if (offset < 0) {
       throw new HadoopIllegalArgumentException(
           "Negative offset is not supported. File: " + src);
@@ -1784,7 +1809,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           "Negative length is not supported. File: " + src);
     }
     final GetBlockLocationsResult ret = getBlockLocationsInt(
-        src, offset, length, needBlockToken);
+        pc, src, offset, length, needBlockToken);
 
     if (checkSafeMode && isInSafeMode()) {
       for (LocatedBlock b : ret.blocks.getLocatedBlocks()) {
@@ -1805,12 +1830,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   private GetBlockLocationsResult getBlockLocationsInt(
-      final String srcArg, long offset, long length, boolean needBlockToken)
+      FSPermissionChecker pc, final String srcArg, long offset, long length,
+      boolean needBlockToken)
       throws IOException {
     String src = srcArg;
-    FSPermissionChecker pc = getPermissionChecker();
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
-    src = dir.resolvePath(pc, src, pathComponents);
+    src = dir.resolvePath(pc, srcArg, pathComponents);
     final INodesInPath iip = dir.getINodesInPath(src, true);
     final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src);
     if (isPermissionEnabled) {
@@ -1846,7 +1871,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean updateAccessTime = isAccessTimeSupported() && !isInSafeMode()
         && !iip.isSnapshot()
         && now > inode.getAccessTime() + getAccessTimePrecision();
-    return new GetBlockLocationsResult(updateAccessTime ? iip : null, blocks);
+    return new GetBlockLocationsResult(updateAccessTime, blocks);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3dd6395b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 23fea12..0cfe31a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -484,7 +484,9 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     FSNamesystem fsn = namenode.getNamesystem();
     fsn.readLock();
     try {
-      blocks = fsn.getBlockLocations(path, 0, fileLen, false, false).blocks;
+      blocks = fsn.getBlockLocations(
+          fsn.getPermissionChecker(), path, 0, fileLen, false, false)
+          .blocks;
     } catch (FileNotFoundException fnfe) {
       blocks = null;
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3dd6395b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index 4c3fa9c..8fe273b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.anyString;
@@ -1154,10 +1155,11 @@ public class TestFsck {
     FSNamesystem fsName = mock(FSNamesystem.class);
     BlockManager blockManager = mock(BlockManager.class);
     DatanodeManager dnManager = mock(DatanodeManager.class);
-    
+
     when(namenode.getNamesystem()).thenReturn(fsName);
-    when(fsName.getBlockLocations(
-        anyString(), anyLong(), anyLong(), anyBoolean(), anyBoolean()))
+    when(fsName.getBlockLocations(any(FSPermissionChecker.class), anyString(),
+                                  anyLong(), anyLong(),
+                                  anyBoolean(), anyBoolean()))
         .thenThrow(new FileNotFoundException());
     when(fsName.getBlockManager()).thenReturn(blockManager);
     when(blockManager.getDatanodeManager()).thenReturn(dnManager);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3dd6395b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
new file mode 100644
index 0000000..a19eb1d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetBlockLocations.java
@@ -0,0 +1,133 @@
+/**
+ * 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 org.apache.commons.io.Charsets;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY;
+import static org.apache.hadoop.util.Time.now;
+import static org.mockito.Mockito.anyLong;
+import static org.mockito.Mockito.anyString;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class TestGetBlockLocations {
+  private static final String FILE_NAME = "foo";
+  private static final String FILE_PATH = "/" + FILE_NAME;
+  private static final long MOCK_INODE_ID = 16386;
+  private static final String RESERVED_PATH =
+      "/.reserved/.inodes/" + MOCK_INODE_ID;
+
+  @Test(timeout = 30000)
+  public void testResolveReservedPath() throws IOException {
+    FSNamesystem fsn = setupFileSystem();
+    FSEditLog editlog = fsn.getEditLog();
+    fsn.getBlockLocations("dummy", RESERVED_PATH, 0, 1024);
+    verify(editlog).logTimes(eq(FILE_PATH), anyLong(), anyLong());
+    fsn.close();
+  }
+
+  @Test(timeout = 30000)
+  public void testGetBlockLocationsRacingWithDelete() throws IOException {
+    FSNamesystem fsn = spy(setupFileSystem());
+    final FSDirectory fsd = fsn.getFSDirectory();
+    FSEditLog editlog = fsn.getEditLog();
+
+    doAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        INodesInPath iip = fsd.getINodesInPath(FILE_PATH, true);
+        FSDirDeleteOp.delete(fsd, iip, new INode.BlocksMapUpdateInfo(),
+                             new ArrayList<INode>(), now());
+        invocation.callRealMethod();
+        return null;
+      }
+    }).when(fsn).writeLock();
+    fsn.getBlockLocations("dummy", RESERVED_PATH, 0, 1024);
+
+    verify(editlog, never()).logTimes(anyString(), anyLong(), anyLong());
+    fsn.close();
+  }
+
+  @Test(timeout = 30000)
+  public void testGetBlockLocationsRacingWithRename() throws IOException {
+    FSNamesystem fsn = spy(setupFileSystem());
+    final FSDirectory fsd = fsn.getFSDirectory();
+    FSEditLog editlog = fsn.getEditLog();
+    final String DST_PATH = "/bar";
+    final boolean[] renamed = new boolean[1];
+
+    doAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        invocation.callRealMethod();
+        if (!renamed[0]) {
+          FSDirRenameOp.renameTo(fsd, fsd.getPermissionChecker(), FILE_PATH,
+                                 DST_PATH, new INode.BlocksMapUpdateInfo(),
+                                 false);
+          renamed[0] = true;
+        }
+        return null;
+      }
+    }).when(fsn).writeLock();
+    fsn.getBlockLocations("dummy", RESERVED_PATH, 0, 1024);
+
+    verify(editlog).logTimes(eq(DST_PATH), anyLong(), anyLong());
+    fsn.close();
+  }
+
+  private static FSNamesystem setupFileSystem() throws IOException {
+    Configuration conf = new Configuration();
+    conf.setLong(DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, 1L);
+    FSEditLog editlog = mock(FSEditLog.class);
+    FSImage image = mock(FSImage.class);
+    when(image.getEditLog()).thenReturn(editlog);
+    final FSNamesystem fsn = new FSNamesystem(conf, image, true);
+
+    final FSDirectory fsd = fsn.getFSDirectory();
+    INodesInPath iip = fsd.getINodesInPath("/", true);
+    PermissionStatus perm = new PermissionStatus(
+        "hdfs", "supergroup",
+        FsPermission.createImmutable((short) 0x1ff));
+    final INodeFile file = new INodeFile(
+        MOCK_INODE_ID, FILE_NAME.getBytes(Charsets.UTF_8),
+        perm, 1, 1, new BlockInfoContiguous[] {}, (short) 1,
+        DFS_BLOCK_SIZE_DEFAULT);
+    fsn.getFSDirectory().addINode(iip, file);
+    return fsn;
+  }
+
+}