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/02 00:35:12 UTC

svn commit: r1498728 - in /hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common: ./ src/main/java/org/apache/hadoop/fs/ src/main/java/org/apache/hadoop/fs/local/

Author: cmccabe
Date: Mon Jul  1 22:35:11 2013
New Revision: 1498728

URL: http://svn.apache.org/r1498728
Log:
HADOOP-9414.  Refactor out FSLinkResolver and relevant helper methods.

Added:
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSLinkResolver.java
Modified:
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
    hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1498728&r1=1498727&r2=1498728&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/CHANGES.txt Mon Jul  1 22:35:11 2013
@@ -34,6 +34,9 @@ Release 2.2.0 - UNRELEASED
     HADOOP-9673.  NetworkTopology: when a node can't be added, print out its
     location for diagnostic purposes.  (Colin Patrick McCabe)
 
+    HADOOP-9414.  Refactor out FSLinkResolver and relevant helper methods.
+    (Andrew Wang via Colin Patrick McCabe)
+
   OPTIMIZATIONS
 
   BUG FIXES

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java?rev=1498728&r1=1498727&r2=1498728&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java Mon Jul  1 22:35:11 2013
@@ -729,6 +729,7 @@ public abstract class AbstractFileSystem
   
   /**
    * Returns true if the file system supports symlinks, false otherwise.
+   * @return true if filesystem supports symlinks
    */
   public boolean supportsSymlinks() {
     return false;
@@ -744,8 +745,9 @@ public abstract class AbstractFileSystem
   }
 
   /**
-   * The specification of this method matches that of  
-   * {@link FileContext#getLinkTarget(Path)};
+   * Partially resolves the path. This is used during symlink resolution in
+   * {@link FSLinkResolver}, and differs from the similarly named method
+   * {@link FileContext#getLinkTarget(Path)}.
    */
   public Path getLinkTarget(final Path f) throws IOException {
     /* We should never get here. Any file system that threw an

Added: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSLinkResolver.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSLinkResolver.java?rev=1498728&view=auto
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSLinkResolver.java (added)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSLinkResolver.java Mon Jul  1 22:35:11 2013
@@ -0,0 +1,118 @@
+/**
+ * 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 java.net.URI;
+
+/**
+ * Class used to perform an operation on and resolve symlinks in a
+ * path. The operation may potentially span multiple file systems.
+ */
+public abstract class FSLinkResolver<T> {
+
+  private static final int MAX_PATH_LINKS = 32;
+
+  /**
+   * See {@link #qualifySymlinkTarget(URI, Path, Path)}
+   */
+  public static Path qualifySymlinkTarget(final AbstractFileSystem pathFS,
+    Path pathWithLink, Path target) {
+    return qualifySymlinkTarget(pathFS.getUri(), pathWithLink, target);
+  }
+
+  /**
+   * See {@link #qualifySymlinkTarget(URI, Path, Path)}
+   */
+  public static Path qualifySymlinkTarget(final FileSystem pathFS,
+      Path pathWithLink, Path target) {
+    return qualifySymlinkTarget(pathFS.getUri(), pathWithLink, target);
+  }
+  
+  /**
+   * Return a fully-qualified version of the given symlink target if it
+   * has no scheme and authority. Partially and fully-qualified paths
+   * are returned unmodified.
+   * @param pathURI URI of the filesystem of pathWithLink
+   * @param pathWithLink Path that contains the symlink
+   * @param target The symlink's absolute target
+   * @return Fully qualified version of the target.
+   */
+  private static Path qualifySymlinkTarget(final URI pathURI,
+      Path pathWithLink, Path target) {
+    // NB: makeQualified uses the target's scheme and authority, if
+    // specified, and the scheme and authority of pathURI, if not.
+    final URI targetUri = target.toUri();
+    final String scheme = targetUri.getScheme();
+    final String auth = targetUri.getAuthority();
+    return (scheme == null && auth == null) ? target.makeQualified(pathURI,
+        pathWithLink.getParent()) : target;
+  }
+
+  // FileContext / AbstractFileSystem resolution methods
+
+  /**
+   * Generic helper function overridden on instantiation to perform a
+   * specific operation on the given file system using the given path
+   * which may result in an UnresolvedLinkException.
+   * @param fs AbstractFileSystem to perform the operation on.
+   * @param p Path given the file system.
+   * @return Generic type determined by the specific implementation.
+   * @throws UnresolvedLinkException If symbolic link <code>path</code> could
+   *           not be resolved
+   * @throws IOException an I/O error occurred
+   */
+  public T next(final AbstractFileSystem fs, final Path p)
+      throws IOException, UnresolvedLinkException {
+    throw new AssertionError("Should not be called without first overriding!");
+  }
+
+  /**
+   * Performs the operation specified by the next function, calling it
+   * repeatedly until all symlinks in the given path are resolved.
+   * @param fc FileContext used to access file systems.
+   * @param path The path to resolve symlinks on.
+   * @return Generic type determined by the implementation of next.
+   * @throws IOException
+   */
+  public T resolve(final FileContext fc, final Path path) throws IOException {
+    int count = 0;
+    T in = null;
+    Path p = path;
+    // NB: More than one AbstractFileSystem can match a scheme, eg 
+    // "file" resolves to LocalFs but could have come by RawLocalFs.
+    AbstractFileSystem fs = fc.getFSofPath(p);
+
+    // Loop until all symlinks are resolved or the limit is reached
+    for (boolean isLink = true; isLink;) {
+      try {
+        in = next(fs, p);
+        isLink = false;
+      } catch (UnresolvedLinkException e) {
+        if (count++ > MAX_PATH_LINKS) {
+          throw new IOException("Possible cyclic loop while " +
+                                "following symbolic link " + path);
+        }
+        // Resolve the first unresolved path component
+        p = FSLinkResolver.qualifySymlinkTarget(fs, p, fs.getLinkTarget(p));
+        fs = fc.getFSofPath(p);
+      }
+    }
+    return in;
+  }
+}

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java?rev=1498728&r1=1498727&r2=1498728&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java Mon Jul  1 22:35:11 2013
@@ -292,17 +292,6 @@ public final class FileContext {
       DELETE_ON_EXIT.clear();
     }
   }
-  
-  /**
-   * Pathnames with scheme and relative path are illegal.
-   * @param path to be checked
-   */
-  private static void checkNotSchemeWithRelative(final Path path) {
-    if (path.toUri().isAbsolute() && !path.isUriPathAbsolute()) {
-      throw new HadoopIllegalArgumentException(
-          "Unsupported name: has scheme but relative path-part");
-    }
-  }
 
   /**
    * Get the file system of supplied path.
@@ -315,13 +304,10 @@ public final class FileContext {
    * @throws IOExcepton If the file system for <code>absOrFqPath</code> could
    *         not be instantiated.
    */
-  private AbstractFileSystem getFSofPath(final Path absOrFqPath)
+  protected AbstractFileSystem getFSofPath(final Path absOrFqPath)
       throws UnsupportedFileSystemException, IOException {
-    checkNotSchemeWithRelative(absOrFqPath);
-    if (!absOrFqPath.isAbsolute() && absOrFqPath.toUri().getScheme() == null) {
-      throw new HadoopIllegalArgumentException(
-          "FileContext Bug: path is relative");
-    }
+    absOrFqPath.checkNotSchemeWithRelative();
+    absOrFqPath.checkNotRelative();
 
     try { 
       // Is it the default FS for this FileContext?
@@ -523,7 +509,7 @@ public final class FileContext {
    *           </ul>
    */
   public void setWorkingDirectory(final Path newWDir) throws IOException {
-    checkNotSchemeWithRelative(newWDir);
+    newWDir.checkNotSchemeWithRelative();
     /* wd is stored as a fully qualified path. We check if the given 
      * path is not relative first since resolve requires and returns 
      * an absolute path.
@@ -1129,26 +1115,6 @@ public final class FileContext {
   }
 
   /**
-   * Return a fully qualified version of the given symlink target if it
-   * has no scheme and authority. Partially and fully qualified paths 
-   * are returned unmodified.
-   * @param pathFS The AbstractFileSystem of the path
-   * @param pathWithLink Path that contains the symlink
-   * @param target The symlink's absolute target
-   * @return Fully qualified version of the target.
-   */
-  private Path qualifySymlinkTarget(final AbstractFileSystem pathFS,
-    Path pathWithLink, Path target) {
-    // NB: makeQualified uses the target's scheme and authority, if
-    // specified, and the scheme and authority of pathFS, if not.
-    final String scheme = target.toUri().getScheme();
-    final String auth   = target.toUri().getAuthority();
-    return (scheme == null && auth == null)
-      ? target.makeQualified(pathFS.getUri(), pathWithLink.getParent())
-      : target;
-  }
-  
-  /**
    * Return a file status object that represents the path. If the path 
    * refers to a symlink then the FileStatus of the symlink is returned.
    * The behavior is equivalent to #getFileStatus() if the underlying
@@ -2166,9 +2132,9 @@ public final class FileContext {
         boolean overwrite) throws AccessControlException,
         FileAlreadyExistsException, FileNotFoundException,
         ParentNotDirectoryException, UnsupportedFileSystemException, 
-	IOException {
-      checkNotSchemeWithRelative(src);
-      checkNotSchemeWithRelative(dst);
+        IOException {
+      src.checkNotSchemeWithRelative();
+      dst.checkNotSchemeWithRelative();
       Path qSrc = makeQualified(src);
       Path qDst = makeQualified(dst);
       checkDest(qSrc.getName(), qDst, overwrite);
@@ -2334,64 +2300,7 @@ public final class FileContext {
     }.resolve(this, absF);
     return result;
   }
-  
-  /**
-   * Class used to perform an operation on and resolve symlinks in a
-   * path. The operation may potentially span multiple file systems.  
-   */
-  protected abstract class FSLinkResolver<T> {
-    // The maximum number of symbolic link components in a path
-    private static final int MAX_PATH_LINKS = 32;
-
-    /**
-     * Generic helper function overridden on instantiation to perform a 
-     * specific operation on the given file system using the given path
-     * which may result in an UnresolvedLinkException. 
-     * @param fs AbstractFileSystem to perform the operation on.
-     * @param p Path given the file system.
-     * @return Generic type determined by the specific implementation.
-     * @throws UnresolvedLinkException If symbolic link <code>path</code> could 
-     *           not be resolved
-     * @throws IOException an I/O error occured
-     */
-    public abstract T next(final AbstractFileSystem fs, final Path p) 
-      throws IOException, UnresolvedLinkException;  
-        
-    /**
-     * Performs the operation specified by the next function, calling it
-     * repeatedly until all symlinks in the given path are resolved.
-     * @param fc FileContext used to access file systems.
-     * @param p The path to resolve symlinks in.
-     * @return Generic type determined by the implementation of next.
-     * @throws IOException
-     */
-    public T resolve(final FileContext fc, Path p) throws IOException {
-      int count = 0;
-      T in = null;
-      Path first = p;
-      // NB: More than one AbstractFileSystem can match a scheme, eg 
-      // "file" resolves to LocalFs but could have come by RawLocalFs.
-      AbstractFileSystem fs = fc.getFSofPath(p);      
 
-      // Loop until all symlinks are resolved or the limit is reached
-      for (boolean isLink = true; isLink;) {
-        try {
-          in = next(fs, p);
-          isLink = false;
-        } catch (UnresolvedLinkException e) {
-          if (count++ > MAX_PATH_LINKS) {
-            throw new IOException("Possible cyclic loop while " +
-                                  "following symbolic link " + first);
-          }
-          // Resolve the first unresolved path component
-          p = qualifySymlinkTarget(fs, p, fs.getLinkTarget(p));
-          fs = fc.getFSofPath(p);
-        }
-      }
-      return in;
-    }
-  }
-  
   /**
    * Get the statistics for a particular file system
    * 

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java?rev=1498728&r1=1498727&r2=1498728&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Path.java Mon Jul  1 22:35:11 2013
@@ -25,6 +25,7 @@ import java.util.regex.Pattern;
 
 import org.apache.avro.reflect.Stringable;
 import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -57,6 +58,32 @@ public class Path implements Comparable 
 
   private URI uri;                                // a hierarchical uri
 
+  /**
+   * Pathnames with scheme and relative path are illegal.
+   * @param path to be checked
+   */
+  void checkNotSchemeWithRelative() {
+    if (toUri().isAbsolute() && !isUriPathAbsolute()) {
+      throw new HadoopIllegalArgumentException(
+          "Unsupported name: has scheme but relative path-part");
+    }
+  }
+
+  void checkNotRelative() {
+    if (!isAbsolute() && toUri().getScheme() == null) {
+      throw new HadoopIllegalArgumentException("Path is relative");
+    }
+  }
+
+  public static Path getPathWithoutSchemeAndAuthority(Path path) {
+    // This code depends on Path.toString() to remove the leading slash before
+    // the drive specification on Windows.
+    Path newPath = path.isUriPathAbsolute() ?
+      new Path(null, null, path.toUri().getPath()) :
+      path;
+    return newPath;
+  }
+
   /** Resolve a child path against a parent path. */
   public Path(String parent, String child) {
     this(new Path(parent), new Path(child));

Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java?rev=1498728&r1=1498727&r2=1498728&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/local/RawLocalFs.java Mon Jul  1 22:35:11 2013
@@ -17,20 +17,20 @@
  */
 package org.apache.hadoop.fs.local;
 
-import java.io.IOException;
-import java.io.File;
 import java.io.FileNotFoundException;
+import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.AbstractFileSystem;
 import org.apache.hadoop.fs.DelegateToFileSystem;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.Shell;
@@ -91,8 +91,8 @@ public class RawLocalFs extends Delegate
     // NB: Use createSymbolicLink in java.nio.file.Path once available
     try {
       Shell.execCommand(Shell.getSymlinkCommand(
-        getPathWithoutSchemeAndAuthority(target).getPath(),
-        getPathWithoutSchemeAndAuthority(link).getPath()));
+        Path.getPathWithoutSchemeAndAuthority(target).toString(),
+        Path.getPathWithoutSchemeAndAuthority(link).toString()));
     } catch (IOException x) {
       throw new IOException("Unable to create symlink: "+x.getMessage());
     }
@@ -175,13 +175,4 @@ public class RawLocalFs extends Delegate
      */
     throw new AssertionError();
   }
-
-  private static File getPathWithoutSchemeAndAuthority(Path path) {
-    Path newPath = path.isUriPathAbsolute() ?
-      new Path(null, null, path.toUri().getPath()) :
-      path;
-
-    // Path.toString() removes leading slash before drive spec on Windows.
-    return new File(newPath.toString());
-  }
 }