Ver código fonte

HDFS-5329. Merging change 1531617 from trunk

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1531618 13f79535-47bb-0310-9956-ffa450edef68
Brandon Li 11 anos atrás
pai
commit
d890e365c8

+ 41 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DirectoryListingStartAfterNotFoundException.java

@@ -0,0 +1,41 @@
+/**
+ * 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.fs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/** 
+ * Thrown when the startAfter can't be found when listing a directory.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS"})
+@InterfaceStability.Stable
+public class DirectoryListingStartAfterNotFoundException extends IOException {
+  private static final long serialVersionUID = 1L;
+
+  public DirectoryListingStartAfterNotFoundException() {
+    super();
+  }
+
+  public DirectoryListingStartAfterNotFoundException(String msg) {
+    super(msg);
+  }
+}

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

@@ -128,6 +128,9 @@ Release 2.2.1 - UNRELEASED
     HDFS-5322. HDFS delegation token not found in cache errors seen on secure HA 
     clusters. (jing9)
 
+    HDFS-5329. Update FSNamesystem#getListing() to handle inode path in startAfter
+    token. (brandonli)
+
 Release 2.2.0 - 2013-10-13
 
   INCOMPATIBLE CHANGES

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -123,6 +123,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FsServerDefaults;
@@ -3897,11 +3898,27 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.READ);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    String startAfterString = new String(startAfter);
     readLock();
     try {
       checkOperation(OperationCategory.READ);
       src = FSDirectory.resolvePath(src, pathComponents, dir);
 
+      // Get file name when startAfter is an INodePath
+      if (FSDirectory.isReservedName(startAfterString)) {
+        byte[][] startAfterComponents = FSDirectory
+            .getPathComponentsForReservedPath(startAfterString);
+        try {
+          String tmp = FSDirectory.resolvePath(src, startAfterComponents, dir);
+          byte[][] regularPath = INode.getPathComponents(tmp);
+          startAfter = regularPath[regularPath.length - 1];
+        } catch (IOException e) {
+          // Possibly the inode is deleted
+          throw new DirectoryListingStartAfterNotFoundException(
+              "Can't find startAfter " + startAfterString);
+        }
+      }
+      
       if (isPermissionEnabled) {
         if (dir.isDir(src)) {
           checkPathAccess(pc, src, FsAction.READ_EXECUTE);

+ 51 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -31,6 +31,7 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -50,6 +51,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -920,7 +922,55 @@ public class TestINodeFile {
       assertTrue(parentId == status.getFileId());
       
     } finally {
-      cluster.shutdown();
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+  
+  @Test
+  public void testFilesInGetListingOps() throws Exception {
+    final Configuration conf = new Configuration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+      cluster.waitActive();
+      final DistributedFileSystem hdfs = cluster.getFileSystem();
+      final FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+
+      hdfs.mkdirs(new Path("/tmp"));
+      DFSTestUtil.createFile(hdfs, new Path("/tmp/f1"), 0, (short) 1, 0);
+      DFSTestUtil.createFile(hdfs, new Path("/tmp/f2"), 0, (short) 1, 0);
+      DFSTestUtil.createFile(hdfs, new Path("/tmp/f3"), 0, (short) 1, 0);
+
+      DirectoryListing dl = cluster.getNameNodeRpc().getListing("/tmp",
+          HdfsFileStatus.EMPTY_NAME, false);
+      assertTrue(dl.getPartialListing().length == 3);
+
+      String f2 = new String("f2");
+      dl = cluster.getNameNodeRpc().getListing("/tmp", f2.getBytes(), false);
+      assertTrue(dl.getPartialListing().length == 1);
+
+      INode f2INode = fsdir.getINode("/tmp/f2");
+      String f2InodePath = "/.reserved/.inodes/" + f2INode.getId();
+      dl = cluster.getNameNodeRpc().getListing("/tmp", f2InodePath.getBytes(),
+          false);
+      assertTrue(dl.getPartialListing().length == 1);
+
+      // Test the deleted startAfter file
+      hdfs.delete(new Path("/tmp/f2"), false);
+      try {
+        dl = cluster.getNameNodeRpc().getListing("/tmp",
+            f2InodePath.getBytes(), false);
+        fail("Didn't get exception for the deleted startAfter token.");
+      } catch (IOException e) {
+        assertTrue(e instanceof DirectoryListingStartAfterNotFoundException);
+      }
+
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
     }
   }
 }