Procházet zdrojové kódy

HADOOP-9418. Add symlink support to DistributedFileSystem (Andrew Wang via Colin Patrick McCabe)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1502373 13f79535-47bb-0310-9956-ffa450edef68
Colin McCabe před 12 roky
rodič
revize
8767e4cde1

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

@@ -263,6 +263,9 @@ Release 2.1.1-beta - UNRELEASED
 
   IMPROVEMENTS
 
+    HADOOP-9418.  Add symlink support to DistributedFileSystem (Andrew Wang via
+    Colin Patrick McCabe)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 12 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -1020,7 +1020,8 @@ public class DFSClient implements java.io.Closeable {
       return namenode.recoverLease(src, clientName);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(FileNotFoundException.class,
-                                     AccessControlException.class);
+                                     AccessControlException.class,
+                                     UnresolvedPathException.class);
     }
   }
 
@@ -2159,7 +2160,11 @@ public class DFSClient implements java.io.Closeable {
    */
   public void allowSnapshot(String snapshotRoot) throws IOException {
     checkOpen();
-    namenode.allowSnapshot(snapshotRoot);
+    try {
+      namenode.allowSnapshot(snapshotRoot);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
   }
   
   /**
@@ -2169,7 +2174,11 @@ public class DFSClient implements java.io.Closeable {
    */
   public void disallowSnapshot(String snapshotRoot) throws IOException {
     checkOpen();
-    namenode.disallowSnapshot(snapshotRoot);
+    try {
+      namenode.disallowSnapshot(snapshotRoot);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException();
+    }
   }
   
   /**

Rozdílová data souboru nebyla zobrazena, protože soubor je příliš velký
+ 655 - 130
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java


+ 11 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java

@@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotException;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
@@ -414,7 +415,11 @@ public class DFSAdmin extends FsShell {
    */
   public void allowSnapshot(String[] argv) throws IOException {   
     DistributedFileSystem dfs = getDFS();
-    dfs.allowSnapshot(new Path(argv[1]));
+    try {
+      dfs.allowSnapshot(new Path(argv[1]));
+    } catch (SnapshotException e) {
+      throw new RemoteException(e.getClass().getName(), e.getMessage());
+    }
     System.out.println("Allowing snaphot on " + argv[1] + " succeeded");
   }
   
@@ -426,7 +431,11 @@ public class DFSAdmin extends FsShell {
    */
   public void disallowSnapshot(String[] argv) throws IOException {  
     DistributedFileSystem dfs = getDFS();
-    dfs.disallowSnapshot(new Path(argv[1]));
+    try {
+      dfs.disallowSnapshot(new Path(argv[1]));
+    } catch (SnapshotException e) {
+      throw new RemoteException(e.getClass().getName(), e.getMessage());
+    }
     System.out.println("Disallowing snaphot on " + argv[1] + " succeeded");
   }
   

+ 107 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfsFileSystem.java

@@ -0,0 +1,107 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+public class TestSymlinkHdfsFileSystem extends TestSymlinkHdfs {
+
+  @BeforeClass
+  public static void testSetup() throws Exception {
+    wrapper = new FileSystemTestWrapper(dfs, "/tmp/TestSymlinkHdfsFileSystem");
+  }
+
+  @Override
+  @Ignore("FileSystem adds missing authority in absolute URIs")
+  @Test(timeout=1000)
+  public void testCreateWithPartQualPathFails() throws IOException {}
+
+  @Ignore("FileSystem#create creates parent directories," +
+      " so dangling links to directories are created")
+  @Override
+  @Test(timeout=1000)
+  public void testCreateFileViaDanglingLinkParent() throws IOException {}
+
+  // Additional tests for DFS-only methods
+
+  @Test(timeout=10000)
+  public void testRecoverLease() throws IOException {
+    Path dir  = new Path(testBaseDir1());
+    Path file = new Path(testBaseDir1(), "file");
+    Path link = new Path(testBaseDir1(), "link");
+    wrapper.setWorkingDirectory(dir);
+    createAndWriteFile(file);
+    wrapper.createSymlink(file, link, false);
+    // Attempt recoverLease through a symlink
+    boolean closed = dfs.recoverLease(link);
+    assertTrue("Expected recoverLease to return true", closed);
+  }
+
+  @Test(timeout=10000)
+  public void testIsFileClosed() throws IOException {
+    Path dir  = new Path(testBaseDir1());
+    Path file = new Path(testBaseDir1(), "file");
+    Path link = new Path(testBaseDir1(), "link");
+    wrapper.setWorkingDirectory(dir);
+    createAndWriteFile(file);
+    wrapper.createSymlink(file, link, false);
+    // Attempt recoverLease through a symlink
+    boolean closed = dfs.isFileClosed(link);
+    assertTrue("Expected isFileClosed to return true", closed);
+  }
+
+  @Test(timeout=10000)
+  public void testConcat() throws Exception {
+    Path dir  = new Path(testBaseDir1());
+    Path link = new Path(testBaseDir1(), "link");
+    Path dir2 = new Path(testBaseDir2());
+    wrapper.createSymlink(dir2, link, false);
+    wrapper.setWorkingDirectory(dir);
+    // Concat with a target and srcs through a link
+    Path target = new Path(link, "target");
+    createAndWriteFile(target);
+    Path[] srcs = new Path[3];
+    for (int i=0; i<srcs.length; i++) {
+      srcs[i] = new Path(link, "src-" + i);
+      createAndWriteFile(srcs[i]);
+    }
+    dfs.concat(target, srcs);
+  }
+
+  @Test(timeout=10000)
+  public void testSnapshot() throws Exception {
+    Path dir  = new Path(testBaseDir1());
+    Path link = new Path(testBaseDir1(), "link");
+    Path dir2 = new Path(testBaseDir2());
+    wrapper.createSymlink(dir2, link, false);
+    wrapper.setWorkingDirectory(dir);
+    dfs.allowSnapshot(link);
+    dfs.disallowSnapshot(link);
+    dfs.allowSnapshot(link);
+    dfs.createSnapshot(link, "mcmillan");
+    dfs.renameSnapshot(link, "mcmillan", "seaborg");
+    dfs.deleteSnapshot(link, "seaborg");
+  }
+}

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java

@@ -157,6 +157,13 @@ public class TestDistributedFileSystem {
     public boolean exists(Path p) {
       return true; // trick out deleteOnExit
     }
+    // Symlink resolution doesn't work with a mock, since it doesn't
+    // have a valid Configuration to resolve paths to the right FileSystem.
+    // Just call the DFSClient directly to register the delete
+    @Override
+    public boolean delete(Path f, final boolean recursive) throws IOException {
+      return dfs.delete(f.toUri().getPath(), recursive);
+    }
   }
 
   @Test

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestNestedSnapshots.java

@@ -135,7 +135,7 @@ public class TestNestedSnapshots {
     try {
       hdfs.disallowSnapshot(rootPath);
       fail("Expect snapshot exception when disallowing snapshot on root again");
-    } catch (RemoteException e) {
+    } catch (SnapshotException e) {
       GenericTestUtils.assertExceptionContains(
           "Root is not a snapshottable directory", e);
     }
@@ -149,16 +149,16 @@ public class TestNestedSnapshots {
     try {
       hdfs.allowSnapshot(rootPath);
       Assert.fail();
-    } catch(RemoteException se) {
+    } catch (SnapshotException se) {
       assertNestedSnapshotException(
-          (SnapshotException) se.unwrapRemoteException(), "subdirectory");
+          se, "subdirectory");
     }
     try {
       hdfs.allowSnapshot(foo);
       Assert.fail();
-    } catch(RemoteException se) {
+    } catch (SnapshotException se) {
       assertNestedSnapshotException(
-          (SnapshotException) se.unwrapRemoteException(), "subdirectory");
+          se, "subdirectory");
     }
 
     final Path sub1Bar = new Path(bar, "sub1");
@@ -167,16 +167,16 @@ public class TestNestedSnapshots {
     try {
       hdfs.allowSnapshot(sub1Bar);
       Assert.fail();
-    } catch(RemoteException se) {
+    } catch (SnapshotException se) {
       assertNestedSnapshotException(
-          (SnapshotException) se.unwrapRemoteException(), "ancestor");
+          se, "ancestor");
     }
     try {
       hdfs.allowSnapshot(sub2Bar);
       Assert.fail();
-    } catch(RemoteException se) {
+    } catch (SnapshotException se) {
       assertNestedSnapshotException(
-          (SnapshotException) se.unwrapRemoteException(), "ancestor");
+          se, "ancestor");
     }
   }
   

Některé soubory nejsou zobrazeny, neboť je v těchto rozdílových datech změněno mnoho souborů