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:27:34 UTC

svn commit: r1392567 - 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:27:33 2012
New Revision: 1392567

URL: http://svn.apache.org/viewvc?rev=1392567&view=rev
Log:
HADOOP-8694. Add support for windows native symbolic links. Contributed by Chuan Liu.

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=1392567&r1=1392566&r2=1392567&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:27:33 2012
@@ -137,3 +137,6 @@ 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=1392567&r1=1392566&r2=1392567&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:27:33 2012
@@ -39,6 +39,13 @@ 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
    * 
@@ -554,115 +561,46 @@ 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
+   * 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.
    * @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{
-    if (Shell.DISABLEWINDOWS_TEMPORARILY) {
-      copyFile(target, linkname);
-      return 0;
-    }
-
-    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));
+    // 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;
     }
-    return returnVal;
+    return shExec.getExitCode();
   }
   
   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=1392567&r1=1392566&r2=1392567&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:27:33 2012
@@ -102,6 +102,12 @@ 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=1392567&r1=1392566&r2=1392567&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:27:33 2012
@@ -45,6 +45,7 @@ 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
@@ -118,7 +119,10 @@ public class TestMRWithDistributedCache 
           context.getConfiguration().get("mapred.job.tracker"))) {
         File symlinkFile = new File("distributed.first.symlink");
         TestCase.assertTrue(symlinkFile.exists());
-        TestCase.assertEquals(1, symlinkFile.length());
+        // 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());
       }
     }
   }

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=1392567&r1=1392566&r2=1392567&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:27:33 2012
@@ -18,6 +18,8 @@
 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;
@@ -328,4 +330,36 @@ 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=1392567&r1=1392566&r2=1392567&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:27:33 2012
@@ -199,12 +199,15 @@ 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;
+//    - If the path is not a directory, change its mode and return.
+//      Symbolic links and junction points are not considered as directories.
 //    - 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;
@@ -215,24 +218,23 @@ static BOOL ChangeFileModeRecursively(__
   DWORD dwRtnCode = ERROR_SUCCESS;
   BOOL ret = FALSE;
 
-  BY_HANDLE_FILE_INFORMATION fileInfo;
-
-  if ((dwRtnCode = GetFileInformationByName(path, &fileInfo)) != ERROR_SUCCESS)
+  if ((dwRtnCode = DirectoryCheck(path, &isDir)) != ERROR_SUCCESS)
+  {
+    ReportErrorCode(L"IsDirectory", dwRtnCode);
+    return FALSE;
+  }
+  if ((dwRtnCode = SymbolicLinkCheck(path, &isSymlink)) != ERROR_SUCCESS)
   {
-    ReportErrorCode(L"GetFileInformationByName", dwRtnCode);
+    ReportErrorCode(L"IsSymbolicLink", dwRtnCode);
     return FALSE;
   }
 
-  if (!IsDirFileInfo(&fileInfo))
+  if (isSymlink || !isDir)
   {
      if (ChangeFileMode(path, mode, actions))
-     {
        return TRUE;
-     }
      else
-     {
        return FALSE;
-     }
   }
 
   if (FAILED(StringCchLengthW(path, STRSAFE_MAX_CCH - 3, &pathSize)))
@@ -499,7 +501,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, &fileInfo);
+      dwRtnCode = GetFileInformationByName(*path, FALSE, &fileInfo);
       if (dwRtnCode != ERROR_SUCCESS)
       {
         ReportErrorCode(L"GetFileInformationByName", dwRtnCode);
@@ -698,7 +700,7 @@ static BOOL ConvertActionsToMask(__in LP
 
   USHORT mode = 0;
 
-  dwErrorCode = GetFileInformationByName(path, &fileInformation);
+  dwErrorCode = GetFileInformationByName(path, FALSE, &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=1392567&r1=1392566&r2=1392567&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:27:33 2012
@@ -61,23 +61,37 @@ 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 = NULL;
+  HANDLE fileHandle = INVALID_HANDLE_VALUE;
+  BOOL isSymlink = FALSE;
+  DWORD dwFlagsAndAttributes = FILE_ATTRIBUTE_NORMAL | FILE_FLAG_BACKUP_SEMANTICS;
   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,
-    FILE_ATTRIBUTE_NORMAL | FILE_FLAG_BACKUP_SEMANTICS,
+    dwFlagsAndAttributes,
     NULL);
   if (fileHandle == INVALID_HANDLE_VALUE)
   {
@@ -199,6 +213,122 @@ 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=1392567&r1=1392566&r2=1392567&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:27:33 2012
@@ -28,6 +28,16 @@
 #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 
@@ -49,6 +59,7 @@ enum UnixAclMask
   UX_U_WRITE   = 0x0080,
   UX_U_READ    = 0x0100,
   UX_DIRECTORY = 0x0200,
+  UX_SYMLINK   = 0x0400,
 };
 
 
@@ -86,10 +97,13 @@ 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,
+DWORD GetFileInformationByName(__in LPCWSTR pathName,  __in BOOL followLink,
   __out LPBY_HANDLE_FILE_INFORMATION lpFileInformation);
 
 DWORD ConvertToLongPath(__in PCWSTR path, __deref_out PWSTR *newPath);
@@ -104,4 +118,8 @@ BOOL FindFileOwnerAndPermission(
   __in LPCWSTR pathName,
   __out_opt LPWSTR *pOwnerName,
   __out_opt LPWSTR *pGroupName,
-  __out_opt PUSHORT pMask);
\ No newline at end of file
+  __out_opt PUSHORT pMask);
+
+DWORD DirectoryCheck(__in LPCWSTR pathName, __out LPBOOL result);
+
+DWORD SymbolicLinkCheck(__in LPCWSTR pathName, __out LPBOOL result);

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=1392567&r1=1392566&r2=1392567&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:27:33 2012
@@ -94,7 +94,7 @@ static DWORD HardlinkStat(__in LPCWSTR f
 
   // Get file information which contains the hard link count
   //
-  dwErrorCode = GetFileInformationByName(longFileName, &fileInformation);
+  dwErrorCode = GetFileInformationByName(longFileName, FALSE, &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=1392567&r1=1392566&r2=1392567&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:27:33 2012
@@ -35,6 +35,8 @@ 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';
@@ -167,6 +169,8 @@ int Ls(int argc, wchar_t *argv[])
 
   LARGE_INTEGER fileSize;
 
+  BOOL isSymlink = FALSE;
+
   int ret = EXIT_FAILURE;
 
   if (argc > 2)
@@ -197,18 +201,25 @@ int Ls(int argc, wchar_t *argv[])
     goto LsEnd;
   }
 
-  dwErrorCode = GetFileInformationByName(longPathName, &fileInformation);
+  dwErrorCode = GetFileInformationByName(longPathName, FALSE, &fileInformation);
   if (dwErrorCode != ERROR_SUCCESS)
   {
     ReportErrorCode(L"GetFileInformationByName", dwErrorCode);
     goto LsEnd;
   }
 
-  if (IsDirFileInfo(&fileInformation))
+  dwErrorCode = SymbolicLinkCheck(pathName, &isSymlink);
+  if (dwErrorCode != ERROR_SUCCESS)
   {
-    accessMask |= UX_DIRECTORY;
+     ReportErrorCode(L"IsSymbolicLink", dwErrorCode);
+     goto LsEnd;
   }
 
+  if (isSymlink)
+    accessMask |= UX_SYMLINK;
+  else if (IsDirFileInfo(&fileInformation))
+    accessMask |= UX_DIRECTORY;
+
   if (!FindFileOwnerAndPermission(longPathName,
     &ownerName, &groupName, &accessMask))
     goto LsEnd;

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=1392567&r1=1392566&r2=1392567&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:27:33 2012
@@ -51,6 +51,10 @@ 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);
@@ -80,18 +84,14 @@ 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,12 +100,20 @@ The available commands and their usages 
   HardlinkUsage();
   fwprintf(stdout, L"\n\n");
 
-  fwprintf(stdout, L"%-15s%s\n\n", L"task", L"Task operations.");
-  TaskUsage();
+  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"\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
+}

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=1392567&r1=1392566&r2=1392567&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:27:33 2012
@@ -135,6 +135,7 @@
     </Link>
   </ItemDefinitionGroup>
   <ItemGroup>
+    <ClCompile Include="symlink.c" />
     <ClCompile Include="systeminfo.c" />
     <ClCompile Include="chmod.c" />
     <ClCompile Include="chown.c" />