Bladeren bron

HADOOP-15465. Deprecate WinUtils#Symlinks by using native java code. Contributed by Giovanni Matteo Fumarola.

Inigo Goiri 7 jaren geleden
bovenliggende
commit
b8d2b09192

+ 24 - 36
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java

@@ -34,8 +34,10 @@ import java.net.URI;
 import java.net.UnknownHostException;
 import java.nio.charset.Charset;
 import java.nio.file.AccessDeniedException;
+import java.nio.file.FileAlreadyExistsException;
 import java.nio.file.FileSystems;
 import java.nio.file.Files;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.List;
@@ -1028,17 +1030,15 @@ public class FileUtil {
   }
 
   /**
-   * Create a soft link between a src and destination
-   * only on a local disk. HDFS does not support this.
-   * On Windows, when symlink creation fails due to security
-   * setting, we will log a warning. The return code in this
-   * case is 2.
+   * Create a soft link between a src and destination only on a local disk. On
+   * Windows, when symlink creation fails due to security setting, we will log a
+   * warning. The return code in this case is 2.
    *
    * @param target the target for symlink
    * @param linkname the symlink
    * @return 0 on success
    */
-  public static int symLink(String target, String linkname) throws IOException{
+  public static int symLink(String target, String linkname) throws IOException {
 
     if (target == null || linkname == null) {
       LOG.warn("Can not create a symLink with a target = " + target
@@ -1053,44 +1053,32 @@ public class FileUtil {
     File linkFile = new File(
         Path.getPathWithoutSchemeAndAuthority(new Path(linkname)).toString());
 
-    String[] cmd = Shell.getSymlinkCommand(
-        targetFile.toString(),
-        linkFile.toString());
-
-    ShellCommandExecutor shExec;
     try {
-      if (Shell.WINDOWS &&
-          linkFile.getParentFile() != null &&
-          !new Path(target).isAbsolute()) {
-        // Relative links on Windows must be resolvable at the time of
-        // creation. To ensure this we run the shell command in the directory
-        // of the link.
-        //
-        shExec = new ShellCommandExecutor(cmd, linkFile.getParentFile());
-      } else {
-        shExec = new ShellCommandExecutor(cmd);
-      }
-      shExec.execute();
-    } catch (Shell.ExitCodeException ec) {
-      int returnVal = ec.getExitCode();
-      if (Shell.WINDOWS && returnVal == SYMLINK_NO_PRIVILEGE) {
-        LOG.warn("Fail to create symbolic links on Windows. "
-            + "The default security settings in Windows disallow non-elevated "
-            + "administrators and all non-administrators from creating symbolic links. "
-            + "This behavior can be changed in the Local Security Policy management console");
-      } else if (returnVal != 0) {
-        LOG.warn("Command '" + StringUtils.join(" ", cmd) + "' failed "
-            + returnVal + " with: " + ec.getMessage());
-      }
-      return returnVal;
+      Files.createSymbolicLink(Paths.get(linkFile.toString()),
+          Paths.get(targetFile.toString()));
+    } catch (SecurityException e3) {
+      LOG.warn("Fail to create symbolic links on Windows. "
+          + "The default security settings in Windows disallow non-elevated "
+          + "administrators and all non-administrators from creating symbolic"
+          + " links. This behavior can be changed in the Local Security Policy"
+          + " management console");
+      return SYMLINK_NO_PRIVILEGE;
+
+    } catch (FileAlreadyExistsException | UnsupportedOperationException e) {
+      LOG.warn("Fail to create symbolic links. ErrorMessage = "
+          + e.getLocalizedMessage());
+      return 1;
+
     } catch (IOException e) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Error while create symlink " + linkname + " to " + target
             + "." + " Exception: " + StringUtils.stringifyException(e));
       }
       throw e;
+
     }
-    return shExec.getExitCode();
+
+    return 0;
   }
 
   /**

+ 0 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java

@@ -925,7 +925,6 @@ public class RawLocalFileSystem extends FileSystem {
     return true;
   }
 
-  @SuppressWarnings("deprecation")
   @Override
   public void createSymlink(Path target, Path link, boolean createParent)
       throws IOException {
@@ -941,7 +940,6 @@ public class RawLocalFileSystem extends FileSystem {
       mkdirs(link.getParent());
     }
 
-    // NB: Use createSymbolicLink in java.nio.file.Path once available
     int result = FileUtil.symLink(target.toString(),
         makeAbsolute(link).toString());
     if (result != 0) {

+ 7 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java

@@ -37,7 +37,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.security.alias.AbstractJavaKeyStoreProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -297,7 +296,13 @@ public abstract class Shell {
         : new String[] { "chown", owner };
   }
 
-  /** Return a command to create symbolic links. */
+  /**
+   * Return a command to create symbolic links.
+   *
+   * Deprecated and likely to be deleted in the near future. Please use
+   * FileUtil.symlink().
+   */
+  @Deprecated
   public static String[] getSymlinkCommand(String target, String link) {
     return WINDOWS ?
        new String[] { getWinUtilsPath(), "symlink", link, target }

+ 6 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@@ -42,7 +43,6 @@ import org.apache.hadoop.net.ServerSocketUtil;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.CompositeService;
 import org.apache.hadoop.util.Shell;
-import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.conf.HAUtil;
@@ -202,14 +202,13 @@ public class MiniYARNCluster extends CompositeService {
       // Guarantee target exists before creating symlink.
       targetWorkDir.mkdirs();
 
-      ShellCommandExecutor shexec = new ShellCommandExecutor(
-        Shell.getSymlinkCommand(targetPath, linkPath));
       try {
-        shexec.execute();
+        FileUtil.symLink(targetPath, linkPath);
       } catch (IOException e) {
-        throw new YarnRuntimeException(String.format(
-          "failed to create symlink from %s to %s, shell output: %s", linkPath,
-          targetPath, shexec.getOutput()), e);
+        throw new YarnRuntimeException(
+            String.format("failed to create symlink from %s to %s.",
+                linkPath, targetPath),
+            e);
       }
 
       this.testWorkDir = link;