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 su...@apache.org on 2012/10/01 22:31:26 UTC

svn commit: r1392570 - in /hadoop/common/branches/branch-1-win: ./ src/core/org/apache/hadoop/fs/ src/core/org/apache/hadoop/util/ src/test/org/apache/hadoop/filecache/ src/test/org/apache/hadoop/fs/ src/winutils/

Author: suresh
Date: Mon Oct  1 20:31:25 2012
New Revision: 1392570

URL: http://svn.apache.org/viewvc?rev=1392570&view=rev
Log:
Reverting r1392567 because this commit missed adding a newly added file

Added:
    hadoop/common/branches/branch-1-win/src/winutils/symlink.c
Modified:
    hadoop/common/branches/branch-1-win/CHANGES.branch-1-win.txt
    hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/fs/FileUtil.java
    hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/Shell.java
    hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/filecache/TestMRWithDistributedCache.java
    hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/fs/TestFileUtil.java
    hadoop/common/branches/branch-1-win/src/winutils/chmod.c
    hadoop/common/branches/branch-1-win/src/winutils/common.c
    hadoop/common/branches/branch-1-win/src/winutils/common.h
    hadoop/common/branches/branch-1-win/src/winutils/hardlink.c
    hadoop/common/branches/branch-1-win/src/winutils/ls.c
    hadoop/common/branches/branch-1-win/src/winutils/main.c
    hadoop/common/branches/branch-1-win/src/winutils/winutils.vcxproj

Modified: hadoop/common/branches/branch-1-win/CHANGES.branch-1-win.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/CHANGES.branch-1-win.txt?rev=1392570&r1=1392569&r2=1392570&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/CHANGES.branch-1-win.txt (original)
+++ hadoop/common/branches/branch-1-win/CHANGES.branch-1-win.txt Mon Oct  1 20:31:25 2012
@@ -137,6 +137,3 @@ BUG FIXES
     HADOOP-8733. Fix the tests TestStreamingTaskLog, TestJvmManager
     and TestLinuxTaskControllerLaunchArgs to not fail on Windows. (Ivan Mitic via
     vinodkv)
-
-    HADOOP-8694. Add support for windows native symbolic links.
-    (Chuan Liu via suresh)

Modified: hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/fs/FileUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/fs/FileUtil.java?rev=1392570&r1=1392569&r2=1392570&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/fs/FileUtil.java (original)
+++ hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/fs/FileUtil.java Mon Oct  1 20:31:25 2012
@@ -39,13 +39,6 @@ import org.apache.hadoop.util.Shell.Shel
 public class FileUtil {
   private static final Log LOG = LogFactory.getLog(FileUtil.class);
 
-  /* The error code is defined in winutils to indicate insufficient
-   * privilege to create symbolic links. This value need to keep in
-   * sync with the constant of the same name in:
-   * "src\winutils\common.h"
-   * */
-  public static final int SYMLINK_NO_PRIVILEGE = 2;
-  
   /**
    * convert an array of FileStatus to an array of Path
    * 
@@ -561,46 +554,115 @@ public class FileUtil {
     }
   }
 
+  //review minwei: temp hack to copy file
+  private static void copyDirectory(String fromFileName, String toFileName)
+      throws IOException {
+
+    File fromFolder = new File(fromFileName);
+    File toFolder = new File(toFileName);
+    if (fromFolder.isFile()) {
+      copyFile(fromFileName, toFileName);
+      return;
+    }
+
+    File[] filelist = fromFolder.listFiles();
+    if (filelist == null) {
+      return;
+    }
+
+    String fromPath = fromFileName;
+    String toPath = toFileName;
+    if (!toFolder.exists())
+      toFolder.mkdirs();
+    for (int i = 0; i < filelist.length; i++) {
+      String subPath = filelist[i].getName();
+      if (filelist[i].isDirectory()) {
+        copyDirectory(fromPath + "/" + subPath, toPath + "/" + subPath);
+      } else {
+        copyFile(fromPath + "/" + subPath, toPath + "/" + subPath);
+      }
+    }
+  }
+
+  private static void copyFile(String fromFileName, String toFileName)
+      throws IOException {
+    File fromFile = new File(fromFileName);
+    File toFile = new File(toFileName);
+
+    if (!fromFile.exists())
+      throw new IOException("FileCopy: " + "no such source file: "
+                            + fromFileName);
+
+    if (fromFile.isDirectory()) {
+      copyDirectory(fromFileName, toFileName);
+      return;
+    }
+
+    if (!fromFile.canRead())
+      throw new IOException("FileCopy: " + "source file is unreadable: "
+                            + fromFileName);
+
+    // Make sure the parent directory exist for the toFileName
+    if (toFile.getParent() != null) {
+      File toFileParentDir = new File(toFile.getParent());
+      if (!toFileParentDir.exists() && !toFileParentDir.mkdirs()) {
+        throw new IOException("FileCopy: failed to create target directory: "
+                              + toFileParentDir.getPath());
+      }
+    }
+
+    InputStream from = null;
+    OutputStream to = null;
+    try {
+      from = new BufferedInputStream(new FileInputStream(fromFile));
+      to = new BufferedOutputStream(new FileOutputStream(toFile));
+      byte[] buffer = new byte[4*1024*1024];
+      int bytesRead;
+
+      while ((bytesRead = from.read(buffer)) != -1)
+        to.write(buffer, 0, bytesRead); // write
+    } finally {
+      if (from != null)
+        try {
+          from.close();
+        } catch (IOException e) {
+          ;
+        }
+      if (to != null)
+        try {
+          to.close();
+        } catch (IOException e) {
+          ;
+        }
+    }
+  }
+  
   /**
    * 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.
+   * only on a local disk. HDFS does not support this
    * @param target the target for symlink 
    * @param linkname the symlink
    * @return value returned by the command
    */
   public static int symLink(String target, String linkname) throws IOException{
-    // Run the input paths through Java's File so that they are converted to the
-    // native OS form. FIXME: Long term fix is to expose symLink API that
-    // accepts File instead of String, as symlinks can only be created on the
-    // local FS.
-    String[] cmd = Shell.getSymlinkCommand(new File(target).getPath(),
-        new File(linkname).getPath());
-    ShellCommandExecutor shExec = new ShellCommandExecutor(cmd);
-    try {
-      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 '" + cmd + "' failed " + returnVal + " with: "
-            + ec.getMessage());
-      }
-      return returnVal;
-    } catch (IOException e) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Error while create symlink " + linkname + " to " + target
-            + "." + " Exception: " + StringUtils.stringifyException(e));
-      }
-      throw e;
+    if (Shell.DISABLEWINDOWS_TEMPORARILY) {
+      copyFile(target, linkname);
+      return 0;
     }
-    return shExec.getExitCode();
+
+    String cmd = "ln -s " + target + " " + linkname;
+    Process p = Runtime.getRuntime().exec(cmd, null);
+    int returnVal = -1;
+    try{
+      returnVal = p.waitFor();
+    } catch(InterruptedException e){
+      //do nothing as of yet
+    }
+    if (returnVal != 0) {
+      LOG.warn("Command '" + cmd + "' failed " + returnVal + 
+               " with: " + copyStderr(p));
+    }
+    return returnVal;
   }
   
   private static String copyStderr(Process p) throws IOException {

Modified: hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/Shell.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/Shell.java?rev=1392570&r1=1392569&r2=1392570&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/Shell.java (original)
+++ hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/Shell.java Mon Oct  1 20:31:25 2012
@@ -102,12 +102,6 @@ abstract public class Shell {
     return (WINDOWS) ? new String[] { WINUTILS, "chown", owner }
                      : new String[] { "chown", owner };
   }
-
-  /** Return a command to create symbolic links */
-  public static String[] getSymlinkCommand(String target, String link) {
-    return WINDOWS ? new String[] { WINUTILS, "symlink", link, target }
-                   : new String[] { "ln", "-s", target, link };
-  }
   
   /** Return a command to execute the given command in OS shell.
    *  On Windows, the passed in groupId can be used to launch

Modified: hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/filecache/TestMRWithDistributedCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/filecache/TestMRWithDistributedCache.java?rev=1392570&r1=1392569&r2=1392570&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/filecache/TestMRWithDistributedCache.java (original)
+++ hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/filecache/TestMRWithDistributedCache.java Mon Oct  1 20:31:25 2012
@@ -45,7 +45,6 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.util.Shell;
 
 /**
  * Tests the use of the
@@ -119,10 +118,7 @@ public class TestMRWithDistributedCache 
           context.getConfiguration().get("mapred.job.tracker"))) {
         File symlinkFile = new File("distributed.first.symlink");
         TestCase.assertTrue(symlinkFile.exists());
-        // Java 6 File#length returns zero for symbolic links on Windows
-        // FIXME: File#length for symbolic links may be due to change in Java 7
-        int expectedValue = Shell.WINDOWS ? 0 : 1;
-        TestCase.assertEquals(expectedValue, symlinkFile.length());
+        TestCase.assertEquals(1, symlinkFile.length());
       }
     }
   }

Modified: hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/fs/TestFileUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/fs/TestFileUtil.java?rev=1392570&r1=1392569&r2=1392570&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/fs/TestFileUtil.java (original)
+++ hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/fs/TestFileUtil.java Mon Oct  1 20:31:25 2012
@@ -18,8 +18,6 @@
 package org.apache.hadoop.fs;
 
 import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collections;
@@ -330,36 +328,4 @@ public class TestFileUtil {
     long du = FileUtil.getDU(TEST_DIR);
     Assert.assertEquals(du, 0);
   }
-
-  @Test
-  public void testSymlink() throws Exception {
-    Assert.assertFalse(del.exists());
-    del.mkdirs();
-
-    byte[] data = "testSymLink".getBytes();
-
-    File file = new File(del, FILE);
-    File link = new File(del, "_link");
-
-    //write some data to the file
-    FileOutputStream os = new FileOutputStream(file);
-    os.write(data);
-    os.close();
-
-    //create the symlink
-    FileUtil.symLink(file.getAbsolutePath(), link.getAbsolutePath());
-
-    //ensure that symlink length is correctly reported by Java
-    Assert.assertEquals(data.length, file.length());
-    Assert.assertEquals(Shell.WINDOWS ? 0 : data.length, link.length());
-
-    //ensure that we can read from link.
-    FileInputStream in = new FileInputStream(link);
-    long len = 0;
-    while (in.read() > 0) {
-      len++;
-    }
-    in.close();
-    Assert.assertEquals(data.length, len);
-  }
 }

Modified: hadoop/common/branches/branch-1-win/src/winutils/chmod.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/winutils/chmod.c?rev=1392570&r1=1392569&r2=1392570&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/winutils/chmod.c (original)
+++ hadoop/common/branches/branch-1-win/src/winutils/chmod.c Mon Oct  1 20:31:25 2012
@@ -199,15 +199,12 @@ static BOOL ChangeFileMode(__in LPCWSTR 
 //
 // Notes:
 //  The recursion works in the following way:
-//    - If the path is not a directory, change its mode and return.
-//      Symbolic links and junction points are not considered as directories.
+//    - If the path is not a directory, change its mode and return;
 //    - Otherwise, call the method on all its children, then change its mode.
 //
 static BOOL ChangeFileModeRecursively(__in LPCWSTR path, __in_opt USHORT mode,
   __in_opt PMODE_CHANGE_ACTION actions)
 {
-  BOOL isDir = FALSE;
-  BOOL isSymlink = FALSE;
   LPWSTR dir = NULL;
 
   size_t pathSize = 0;
@@ -218,23 +215,24 @@ static BOOL ChangeFileModeRecursively(__
   DWORD dwRtnCode = ERROR_SUCCESS;
   BOOL ret = FALSE;
 
-  if ((dwRtnCode = DirectoryCheck(path, &isDir)) != ERROR_SUCCESS)
-  {
-    ReportErrorCode(L"IsDirectory", dwRtnCode);
-    return FALSE;
-  }
-  if ((dwRtnCode = SymbolicLinkCheck(path, &isSymlink)) != ERROR_SUCCESS)
+  BY_HANDLE_FILE_INFORMATION fileInfo;
+
+  if ((dwRtnCode = GetFileInformationByName(path, &fileInfo)) != ERROR_SUCCESS)
   {
-    ReportErrorCode(L"IsSymbolicLink", dwRtnCode);
+    ReportErrorCode(L"GetFileInformationByName", dwRtnCode);
     return FALSE;
   }
 
-  if (isSymlink || !isDir)
+  if (!IsDirFileInfo(&fileInfo))
   {
      if (ChangeFileMode(path, mode, actions))
+     {
        return TRUE;
+     }
      else
+     {
        return FALSE;
+     }
   }
 
   if (FAILED(StringCchLengthW(path, STRSAFE_MAX_CCH - 3, &pathSize)))
@@ -501,7 +499,7 @@ static BOOL ParseCommandLineArguments(__
       // Check if the given path name is a file or directory
       // Only set recursive flag if the given path is a directory
       //
-      dwRtnCode = GetFileInformationByName(*path, FALSE, &fileInfo);
+      dwRtnCode = GetFileInformationByName(*path, &fileInfo);
       if (dwRtnCode != ERROR_SUCCESS)
       {
         ReportErrorCode(L"GetFileInformationByName", dwRtnCode);
@@ -700,7 +698,7 @@ static BOOL ConvertActionsToMask(__in LP
 
   USHORT mode = 0;
 
-  dwErrorCode = GetFileInformationByName(path, FALSE, &fileInformation);
+  dwErrorCode = GetFileInformationByName(path, &fileInformation);
   if (dwErrorCode != ERROR_SUCCESS)
   {
     ReportErrorCode(L"GetFileInformationByName", dwErrorCode);

Modified: hadoop/common/branches/branch-1-win/src/winutils/common.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/winutils/common.c?rev=1392570&r1=1392569&r2=1392570&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/winutils/common.c (original)
+++ hadoop/common/branches/branch-1-win/src/winutils/common.c Mon Oct  1 20:31:25 2012
@@ -61,37 +61,23 @@ const ACCESS_MASK WinMasks[WIN_MASKS_TOT
 //  error code: otherwise
 //
 // Notes:
-//  If followLink parameter is set to TRUE, we will follow the symbolic link
-//  or junction point to get the target file information. Otherwise, the
-//  information for the symbolic link or junction point is retrieved.
 //
 DWORD GetFileInformationByName(
   __in LPCWSTR pathName,
-  __in BOOL followLink,
   __out LPBY_HANDLE_FILE_INFORMATION lpFileInformation)
 {
-  HANDLE fileHandle = INVALID_HANDLE_VALUE;
-  BOOL isSymlink = FALSE;
-  DWORD dwFlagsAndAttributes = FILE_ATTRIBUTE_NORMAL | FILE_FLAG_BACKUP_SEMANTICS;
+  HANDLE fileHandle = NULL;
   DWORD dwErrorCode = ERROR_SUCCESS;
 
   assert(lpFileInformation != NULL);
 
-  if (!followLink)
-  {
-    if ((dwErrorCode = SymbolicLinkCheck(pathName, &isSymlink)) != ERROR_SUCCESS)
-      return dwErrorCode;
-    if (isSymlink)
-      dwFlagsAndAttributes |= FILE_FLAG_OPEN_REPARSE_POINT;
-  }
-
   fileHandle = CreateFileW(
     pathName,
     FILE_READ_ATTRIBUTES,
     FILE_SHARE_READ,
     NULL,
     OPEN_EXISTING,
-    dwFlagsAndAttributes,
+    FILE_ATTRIBUTE_NORMAL | FILE_FLAG_BACKUP_SEMANTICS,
     NULL);
   if (fileHandle == INVALID_HANDLE_VALUE)
   {
@@ -213,122 +199,6 @@ BOOL IsDirFileInfo(const BY_HANDLE_FILE_
 }
 
 //----------------------------------------------------------------------------
-// Function: CheckFileAttributes
-//
-// Description:
-//	Check if the given file has all the given attribute(s)
-//
-// Returns:
-//	ERROR_SUCCESS on success
-//  error code otherwise
-//
-// Notes:
-//
-static DWORD FileAttributesCheck(
-  __in LPCWSTR path, __in DWORD attr, __out PBOOL res)
-{
-  DWORD attrs = INVALID_FILE_ATTRIBUTES;
-  *res = FALSE;
-  if ((attrs = GetFileAttributes(path)) != INVALID_FILE_ATTRIBUTES)
-    *res = ((attrs & attr) == attr);
-  else
-    return GetLastError();
-  return ERROR_SUCCESS;
-}
-
-//----------------------------------------------------------------------------
-// Function: IsDirectory
-//
-// Description:
-//	Check if the given file is a directory
-//
-// Returns:
-//	ERROR_SUCCESS on success
-//  error code otherwise
-//
-// Notes:
-//
-DWORD DirectoryCheck(__in LPCWSTR pathName, __out PBOOL res)
-{
-  return FileAttributesCheck(pathName, FILE_ATTRIBUTE_DIRECTORY, res);
-}
-
-//----------------------------------------------------------------------------
-// Function: IsReparsePoint
-//
-// Description:
-//	Check if the given file is a reparse point
-//
-// Returns:
-//	ERROR_SUCCESS on success
-//  error code otherwise
-//
-// Notes:
-//
-static DWORD ReparsePointCheck(__in LPCWSTR pathName, __out PBOOL res)
-{
-  return FileAttributesCheck(pathName, FILE_ATTRIBUTE_REPARSE_POINT, res);
-}
-
-//----------------------------------------------------------------------------
-// Function: CheckReparseTag
-//
-// Description:
-//	Check if the given file is a reparse point of the given tag.
-//
-// Returns:
-//	ERROR_SUCCESS on success
-//  error code otherwise
-//
-// Notes:
-//
-static DWORD ReparseTagCheck(__in LPCWSTR path, __in DWORD tag, __out PBOOL res)
-{
-  BOOL isReparsePoint = FALSE;
-  HANDLE hFind = INVALID_HANDLE_VALUE;
-  WIN32_FIND_DATA findData;
-  DWORD dwRtnCode;
-
-  if ((dwRtnCode = ReparsePointCheck(path, &isReparsePoint)) != ERROR_SUCCESS)
-    return dwRtnCode;
-
-  if (!isReparsePoint)
-  {
-    *res = FALSE;
-  }
-  else
-  {
-    if ((hFind = FindFirstFile(path, &findData)) == INVALID_HANDLE_VALUE)
-    {
-      return GetLastError();
-    }
-    else
-    {
-      *res = (findData.dwReserved0 == tag);
-      FindClose(hFind);
-    }
-  }
-  return ERROR_SUCCESS;
-}
-
-//----------------------------------------------------------------------------
-// Function: IsSymbolicLink
-//
-// Description:
-//	Check if the given file is a symbolic link.
-//
-// Returns:
-//	ERROR_SUCCESS on success
-//  error code otherwise
-//
-// Notes:
-//
-DWORD SymbolicLinkCheck(__in LPCWSTR pathName, __out PBOOL res)
-{
-  return ReparseTagCheck(pathName, IO_REPARSE_TAG_SYMLINK, res);
-}
-
-//----------------------------------------------------------------------------
 // Function: GetSidFromAcctNameW
 //
 // Description:

Modified: hadoop/common/branches/branch-1-win/src/winutils/common.h
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/winutils/common.h?rev=1392570&r1=1392569&r2=1392570&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/winutils/common.h (original)
+++ hadoop/common/branches/branch-1-win/src/winutils/common.h Mon Oct  1 20:31:25 2012
@@ -28,16 +28,6 @@
 #include <accctrl.h>
 #include <tchar.h>
 #include <strsafe.h>
-enum EXIT_CODE
-{
-  /* Common success exit code shared among all utilities */
-  SUCCESS = EXIT_SUCCESS,
-  /* Generic failure exit code share among all utilities */
-  FAILURE = EXIT_FAILURE,
-  /* Failure code indicates the user does not privilege to create symlinks */
-  SYMLINK_NO_PRIVILEGE = 2,
-};
-
 
 /*
  * The array of 12 months' three-letter abbreviations 
@@ -59,7 +49,6 @@ enum UnixAclMask
   UX_U_WRITE   = 0x0080,
   UX_U_READ    = 0x0100,
   UX_DIRECTORY = 0x0200,
-  UX_SYMLINK   = 0x0400,
 };
 
 
@@ -97,13 +86,10 @@ void HardlinkUsage();
 int Task(int argc, wchar_t *argv[]);
 void TaskUsage();
 
-int Symlink(int argc, wchar_t *argv[]);
-void SymlinkUsage();
-
 int SystemInfo();
 void SystemInfoUsage();
 
-DWORD GetFileInformationByName(__in LPCWSTR pathName,  __in BOOL followLink,
+DWORD GetFileInformationByName(__in LPCWSTR pathName,
   __out LPBY_HANDLE_FILE_INFORMATION lpFileInformation);
 
 DWORD ConvertToLongPath(__in PCWSTR path, __deref_out PWSTR *newPath);
@@ -118,8 +104,4 @@ BOOL FindFileOwnerAndPermission(
   __in LPCWSTR pathName,
   __out_opt LPWSTR *pOwnerName,
   __out_opt LPWSTR *pGroupName,
-  __out_opt PUSHORT pMask);
-
-DWORD DirectoryCheck(__in LPCWSTR pathName, __out LPBOOL result);
-
-DWORD SymbolicLinkCheck(__in LPCWSTR pathName, __out LPBOOL result);
+  __out_opt PUSHORT pMask);
\ No newline at end of file

Modified: hadoop/common/branches/branch-1-win/src/winutils/hardlink.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/winutils/hardlink.c?rev=1392570&r1=1392569&r2=1392570&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/winutils/hardlink.c (original)
+++ hadoop/common/branches/branch-1-win/src/winutils/hardlink.c Mon Oct  1 20:31:25 2012
@@ -94,7 +94,7 @@ static DWORD HardlinkStat(__in LPCWSTR f
 
   // Get file information which contains the hard link count
   //
-  dwErrorCode = GetFileInformationByName(longFileName, FALSE, &fileInformation);
+  dwErrorCode = GetFileInformationByName(longFileName, &fileInformation);
   if (dwErrorCode != ERROR_SUCCESS)
   {
     goto HardlinkStatExit;

Modified: hadoop/common/branches/branch-1-win/src/winutils/ls.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/winutils/ls.c?rev=1392570&r1=1392569&r2=1392570&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/winutils/ls.c (original)
+++ hadoop/common/branches/branch-1-win/src/winutils/ls.c Mon Oct  1 20:31:25 2012
@@ -35,8 +35,6 @@ static BOOL GetMaskString(USHORT accessM
 
   if ((accessMask & UX_DIRECTORY) == UX_DIRECTORY)
     maskString[0] = L'd';
-  else if ((accessMask & UX_SYMLINK) == UX_SYMLINK)
-    maskString[0] = L'l';
 
   if ((accessMask & UX_U_READ) == UX_U_READ)
     maskString[1] = L'r';
@@ -169,8 +167,6 @@ int Ls(int argc, wchar_t *argv[])
 
   LARGE_INTEGER fileSize;
 
-  BOOL isSymlink = FALSE;
-
   int ret = EXIT_FAILURE;
 
   if (argc > 2)
@@ -201,24 +197,17 @@ int Ls(int argc, wchar_t *argv[])
     goto LsEnd;
   }
 
-  dwErrorCode = GetFileInformationByName(longPathName, FALSE, &fileInformation);
+  dwErrorCode = GetFileInformationByName(longPathName, &fileInformation);
   if (dwErrorCode != ERROR_SUCCESS)
   {
     ReportErrorCode(L"GetFileInformationByName", dwErrorCode);
     goto LsEnd;
   }
 
-  dwErrorCode = SymbolicLinkCheck(pathName, &isSymlink);
-  if (dwErrorCode != ERROR_SUCCESS)
+  if (IsDirFileInfo(&fileInformation))
   {
-     ReportErrorCode(L"IsSymbolicLink", dwErrorCode);
-     goto LsEnd;
-  }
-
-  if (isSymlink)
-    accessMask |= UX_SYMLINK;
-  else if (IsDirFileInfo(&fileInformation))
     accessMask |= UX_DIRECTORY;
+  }
 
   if (!FindFileOwnerAndPermission(longPathName,
     &ownerName, &groupName, &accessMask))

Modified: hadoop/common/branches/branch-1-win/src/winutils/main.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/winutils/main.c?rev=1392570&r1=1392569&r2=1392570&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/winutils/main.c (original)
+++ hadoop/common/branches/branch-1-win/src/winutils/main.c Mon Oct  1 20:31:25 2012
@@ -51,10 +51,6 @@ int wmain(int argc, wchar_t* argv[])
   {
     return Hardlink(argc - 1, argv + 1);
   }
-  else if (wcscmp(L"symlink", cmd) == 0)
-  {
-    return Symlink(argc - 1, argv + 1);
-  }
   else if (wcscmp(L"task", cmd) == 0)
   {
     return Task(argc - 1, argv + 1);
@@ -84,14 +80,18 @@ static void Usage(LPCWSTR program)
 Provide basic command line utilities for Hadoop on Windows.\n\n\
 The available commands and their usages are:\n\n", program);
 
+  fwprintf(stdout, L"%-15s%s\n\n", L"ls", L"List file information.");
+  LsUsage(L"ls");
+  fwprintf(stdout, L"\n\n");
+
   fwprintf(stdout, L"%-15s%s\n\n", L"chmod", L"Change file mode bits.");
   ChmodUsage(L"chmod");
   fwprintf(stdout, L"\n\n");
-
+    
   fwprintf(stdout, L"%-15s%s\n\n", L"chown", L"Change file owner.");
   ChownUsage(L"chown");
   fwprintf(stdout, L"\n\n");
-
+  
   fwprintf(stdout, L"%-15s%s\n\n", L"groups", L"List user groups.");
   GroupsUsage(L"groups");
   fwprintf(stdout, L"\n\n");
@@ -100,20 +100,12 @@ The available commands and their usages 
   HardlinkUsage();
   fwprintf(stdout, L"\n\n");
 
-  fwprintf(stdout, L"%-15s%s\n\n", L"ls", L"List file information.");
-  LsUsage(L"ls");
-  fwprintf(stdout, L"\n\n");
- 
-  fwprintf(stdout, L"%-10s%s\n\n", L"symlink", L"Create a symbolic link.");
-  SymlinkUsage();
+  fwprintf(stdout, L"%-15s%s\n\n", L"task", L"Task operations.");
+  TaskUsage();
   fwprintf(stdout, L"\n\n");
 
   fwprintf(stdout, L"%-15s%s\n\n", L"systeminfo", L"System information.");
   SystemInfoUsage();
 
-  fwprintf(stdout, L"%-15s%s\n\n", L"task", L"Task operations.");
-  TaskUsage();
-  fwprintf(stdout, L"\n\n");
-
   fwprintf(stdout, L"\n\n");
-}
+}
\ No newline at end of file

Added: hadoop/common/branches/branch-1-win/src/winutils/symlink.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/winutils/symlink.c?rev=1392570&view=auto
==============================================================================
--- hadoop/common/branches/branch-1-win/src/winutils/symlink.c (added)
+++ hadoop/common/branches/branch-1-win/src/winutils/symlink.c Mon Oct  1 20:31:25 2012
@@ -0,0 +1,160 @@
+/**
+ * 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.
+ */
+
+#include "common.h"
+
+//----------------------------------------------------------------------------
+// Function: EnablePrivilege
+//
+// Description:
+//	Check if the process has the given privilege. If yes, enable the privilege
+//  to the process's access token.
+//
+// Returns:
+//	TRUE: on success
+//
+// Notes:
+//
+static BOOL EnablePrivilege(__in LPCWSTR privilegeName)
+{
+  HANDLE hToken;
+  TOKEN_PRIVILEGES tp;
+  DWORD dwErrCode;
+
+  if (!OpenProcessToken(GetCurrentProcess(),
+    TOKEN_ADJUST_PRIVILEGES | TOKEN_QUERY, &hToken))
+  {
+    ReportErrorCode(L"OpenProcessToken", GetLastError());
+    return FALSE;
+  }
+
+  tp.PrivilegeCount = 1;
+  if (!LookupPrivilegeValueW(NULL,
+    privilegeName, &(tp.Privileges[0].Luid)))
+  {
+    ReportErrorCode(L"LookupPrivilegeValue", GetLastError());
+    CloseHandle(hToken);
+    return FALSE;
+  }
+  tp.Privileges[0].Attributes = SE_PRIVILEGE_ENABLED;
+
+  // As stated on MSDN, we need to use GetLastError() to check if
+  // AdjustTokenPrivileges() adjusted all of the specified privileges.
+  //
+  AdjustTokenPrivileges(hToken, FALSE, &tp, 0, NULL, NULL);
+  dwErrCode = GetLastError();
+  CloseHandle(hToken);
+
+  return dwErrCode == ERROR_SUCCESS;
+}
+
+//----------------------------------------------------------------------------
+// Function: Symlink
+//
+// Description:
+//	The main method for symlink command
+//
+// Returns:
+//	0: on success
+//
+// Notes:
+//
+int Symlink(int argc, wchar_t *argv[])
+{
+  PWSTR longLinkName = NULL;
+  PWSTR longFileName = NULL;
+  DWORD dwErrorCode = ERROR_SUCCESS;
+
+  BOOL isDir = FALSE;
+
+  DWORD dwRtnCode = ERROR_SUCCESS;
+  DWORD dwFlag = 0;
+
+  int ret = SUCCESS;
+
+  if (argc != 3)
+  {
+    SymlinkUsage();
+    return FAILURE;
+  }
+
+  dwErrorCode = ConvertToLongPath(argv[1], &longLinkName);
+  if (dwErrorCode != ERROR_SUCCESS)
+  {
+    ret = FAILURE;
+    goto SymlinkEnd;
+  }
+  dwErrorCode = ConvertToLongPath(argv[2], &longFileName);
+  if (dwErrorCode != ERROR_SUCCESS)
+  {
+    ret = FAILURE;
+    goto SymlinkEnd;
+  }
+
+  // Check if the the process's access token has the privilege to create
+  // symbolic links. Without this step, the call to CreateSymbolicLink() from
+  // users have the privilege to create symbolic links will still succeed.
+  // This is just an additional step to do the privilege check by not using
+  // error code from CreateSymbolicLink() method.
+  //
+  if (!EnablePrivilege(L"SeCreateSymbolicLinkPrivilege"))
+  {
+    fwprintf(stderr,
+      L"No privilege to create symbolic links.\n");
+    ret = SYMLINK_NO_PRIVILEGE;
+    goto SymlinkEnd;
+  }
+
+  if ((dwRtnCode = DirectoryCheck(longFileName, &isDir)) != ERROR_SUCCESS)
+  {
+    ReportErrorCode(L"DirectoryCheck", dwRtnCode);
+    ret = FAILURE;
+    goto SymlinkEnd;
+  }
+
+  if (isDir)
+    dwFlag = SYMBOLIC_LINK_FLAG_DIRECTORY;
+
+  if (!CreateSymbolicLinkW(longLinkName, longFileName, dwFlag))
+  {
+    ReportErrorCode(L"CreateSymbolicLink", GetLastError());
+    ret = FAILURE;
+    goto SymlinkEnd;
+  }
+
+SymlinkEnd:
+  LocalFree(longLinkName);
+  LocalFree(longFileName);
+  return ret;
+}
+
+void SymlinkUsage()
+{
+    fwprintf(stdout, L"\
+Usage: symlink [LINKNAME] [FILENAME]\n\
+Creates a symbolic link\n\
+\n\
+0 is returned on success.\n\
+2 is returned if the user does no have privilege to create symbolic links.\n\
+1 is returned for all other errors.\n\
+\n\
+The default security settings in Windows disallow non-elevated administrators\n\
+and all non-administrators from creating symbolic links. The security settings\n\
+for symbolic links can be changed in the Local Security Policy management\n\
+console.\n");
+}
+

Modified: hadoop/common/branches/branch-1-win/src/winutils/winutils.vcxproj
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/winutils/winutils.vcxproj?rev=1392570&r1=1392569&r2=1392570&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/winutils/winutils.vcxproj (original)
+++ hadoop/common/branches/branch-1-win/src/winutils/winutils.vcxproj Mon Oct  1 20:31:25 2012
@@ -135,7 +135,6 @@
     </Link>
   </ItemDefinitionGroup>
   <ItemGroup>
-    <ClCompile Include="symlink.c" />
     <ClCompile Include="systeminfo.c" />
     <ClCompile Include="chmod.c" />
     <ClCompile Include="chown.c" />