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 cm...@apache.org on 2013/07/11 20:10:47 UTC

svn commit: r1502307 - in /hadoop/common/trunk/hadoop-common-project/hadoop-common/src: main/java/org/apache/hadoop/fs/ test/java/org/apache/hadoop/fs/

Author: cmccabe
Date: Thu Jul 11 18:10:46 2013
New Revision: 1502307

URL: http://svn.apache.org/r1502307
Log:
HADOOP-9417.  Support for symlink resolution in LocalFileSystem / RawLocalFileSystem.  (Andrew Wang via Colin Patrick McCabe)

Added:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFSFileSystem.java
Modified:
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
    hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestWrapper.java

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java?rev=1502307&r1=1502306&r2=1502307&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java Thu Jul 11 18:10:46 2013
@@ -18,9 +18,10 @@
 
 package org.apache.hadoop.fs;
 
-import java.io.*;
+import java.io.File;
+import java.io.IOException;
 import java.net.URI;
-import java.util.*;
+import java.util.Random;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -142,4 +143,25 @@ public class LocalFileSystem extends Che
     }
     return false;
   }
+
+  @Override
+  public boolean supportsSymlinks() {
+    return true;
+  }
+
+  @Override
+  public void createSymlink(Path target, Path link, boolean createParent)
+      throws IOException {
+    fs.createSymlink(target, link, createParent);
+  }
+
+  @Override
+  public FileStatus getFileLinkStatus(final Path f) throws IOException {
+    return fs.getFileLinkStatus(f);
+  }
+
+  @Override
+  public Path getLinkTarget(Path f) throws IOException {
+    return fs.getLinkTarget(f);
+  }
 }

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java?rev=1502307&r1=1502306&r2=1502307&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java Thu Jul 11 18:10:46 2013
@@ -437,7 +437,6 @@ public class RawLocalFileSystem extends 
   public void setWorkingDirectory(Path newDir) {
     workingDir = makeAbsolute(newDir);
     checkPath(workingDir);
-    
   }
   
   @Override
@@ -641,4 +640,109 @@ public class RawLocalFileSystem extends 
     }
   }
 
+  @Override
+  public boolean supportsSymlinks() {
+    return true;
+  }
+
+  @Override
+  public void createSymlink(Path target, Path link, boolean createParent)
+      throws IOException {
+    final String targetScheme = target.toUri().getScheme();
+    if (targetScheme != null && !"file".equals(targetScheme)) {
+      throw new IOException("Unable to create symlink to non-local file "+
+                            "system: "+target.toString());
+    }
+    if (createParent) {
+      mkdirs(link.getParent());
+    }
+    // NB: Use createSymbolicLink in java.nio.file.Path once available
+    try {
+      Shell.execCommand(Shell.getSymlinkCommand(
+        Path.getPathWithoutSchemeAndAuthority(target).toString(),
+        Path.getPathWithoutSchemeAndAuthority(makeAbsolute(link)).toString()));
+    } catch (IOException x) {
+      throw new IOException("Unable to create symlink: "+x.getMessage());
+    }
+  }
+
+  /**
+   * Returns the target of the given symlink. Returns the empty string if
+   * the given path does not refer to a symlink or there is an error
+   * accessing the symlink.
+   */
+  private String readLink(Path p) {
+    /* NB: Use readSymbolicLink in java.nio.file.Path once available. Could
+     * use getCanonicalPath in File to get the target of the symlink but that
+     * does not indicate if the given path refers to a symlink.
+     */
+    try {
+      final String path = p.toUri().getPath();
+      return Shell.execCommand(Shell.READ_LINK_COMMAND, path).trim();
+    } catch (IOException x) {
+      return "";
+    }
+  }
+
+  /**
+   * Return a FileStatus representing the given path. If the path refers
+   * to a symlink return a FileStatus representing the link rather than
+   * the object the link refers to.
+   */
+  @Override
+  public FileStatus getFileLinkStatus(final Path f) throws IOException {
+    FileStatus fi = getFileLinkStatusInternal(f);
+    // getFileLinkStatus is supposed to return a symlink with a
+    // qualified path
+    if (fi.isSymlink()) {
+      Path targetQual = FSLinkResolver.qualifySymlinkTarget(this.getUri(),
+          fi.getPath(), fi.getSymlink());
+      fi.setSymlink(targetQual);
+    }
+    return fi;
+  }
+
+  private FileStatus getFileLinkStatusInternal(final Path f) throws IOException {
+    String target = readLink(f);
+
+    try {
+      FileStatus fs = getFileStatus(f);
+      // If f refers to a regular file or directory
+      if (target.isEmpty()) {
+        return fs;
+      }
+      // Otherwise f refers to a symlink
+      return new FileStatus(fs.getLen(),
+          false,
+          fs.getReplication(),
+          fs.getBlockSize(),
+          fs.getModificationTime(),
+          fs.getAccessTime(),
+          fs.getPermission(),
+          fs.getOwner(),
+          fs.getGroup(),
+          new Path(target),
+          f);
+    } catch (FileNotFoundException e) {
+      /* The exists method in the File class returns false for dangling
+       * links so we can get a FileNotFoundException for links that exist.
+       * It's also possible that we raced with a delete of the link. Use
+       * the readBasicFileAttributes method in java.nio.file.attributes
+       * when available.
+       */
+      if (!target.isEmpty()) {
+        return new FileStatus(0, false, 0, 0, 0, 0, FsPermission.getDefault(),
+            "", "", new Path(target), f);
+      }
+      // f refers to a file or directory that does not exist
+      throw e;
+    }
+  }
+
+  @Override
+  public Path getLinkTarget(Path f) throws IOException {
+    FileStatus fi = getFileLinkStatusInternal(f);
+    // return an unqualified symlink target
+    return fi.getSymlink();
+  }
 }

Modified: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestWrapper.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestWrapper.java?rev=1502307&r1=1502306&r2=1502307&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestWrapper.java (original)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemTestWrapper.java Thu Jul 11 18:10:46 2013
@@ -206,14 +206,13 @@ public final class FileSystemTestWrapper
     return fs.makeQualified(path);
   }
 
+  @SuppressWarnings("deprecation")
   @Override
   public void mkdir(Path dir, FsPermission permission, boolean createParent)
       throws AccessControlException, FileAlreadyExistsException,
       FileNotFoundException, ParentNotDirectoryException,
       UnsupportedFileSystemException, IOException {
-    // Note that there is no "mkdir" in FileSystem, it always does
-    // "mkdir -p" (creating parent directories).
-    fs.mkdirs(dir, permission);
+    fs.primitiveMkdir(dir, permission, createParent);
   }
 
   @Override

Added: hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFSFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFSFileSystem.java?rev=1502307&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFSFileSystem.java (added)
+++ hadoop/common/trunk/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFSFileSystem.java Thu Jul 11 18:10:46 2013
@@ -0,0 +1,57 @@
+/**
+ * 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.conf.Configuration;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+
+public class TestSymlinkLocalFSFileSystem extends TestSymlinkLocalFS {
+
+  @BeforeClass
+  public static void testSetup() throws Exception {
+    FileSystem filesystem = FileSystem.getLocal(new Configuration());
+    wrapper = new FileSystemTestWrapper(filesystem);
+  }
+
+  @Ignore("RawLocalFileSystem#mkdir does not treat existence of directory" +
+      " as an error")
+  @Override
+  @Test(timeout=1000)
+  public void testMkdirExistingLink() throws IOException {}
+
+  @Ignore("FileSystem#create defaults to creating parents," +
+      " throwing an IOException instead of FileNotFoundException")
+  @Override
+  @Test(timeout=1000)
+  public void testCreateFileViaDanglingLinkParent() throws IOException {}
+
+  @Ignore("RawLocalFileSystem does not throw an exception if the path" +
+      " already exists")
+  @Override
+  @Test(timeout=1000)
+  public void testCreateFileDirExistingLink() throws IOException {}
+  
+  @Ignore("ChecksumFileSystem does not support append")
+  @Override
+  @Test(timeout=1000)
+  public void testAccessFileViaInterSymlinkAbsTarget() throws IOException {}
+}