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 iv...@apache.org on 2013/08/11 05:17:22 UTC

svn commit: r1512864 - in /hadoop/common/branches/branch-1-win: ./ src/contrib/failmon/src/java/org/apache/hadoop/contrib/failmon/ src/contrib/streaming/src/java/org/apache/hadoop/streaming/ src/core/org/apache/hadoop/fs/ src/core/org/apache/hadoop/io/...

Author: ivanmi
Date: Sun Aug 11 03:17:21 2013
New Revision: 1512864

URL: http://svn.apache.org/r1512864
Log:
HADOOP-9551. Backport common utils introduced with HADOOP-9413 to branch-1-win. Contributed by Ivan Mitic.

Modified:
    hadoop/common/branches/branch-1-win/CHANGES.branch-1-win.txt
    hadoop/common/branches/branch-1-win/src/contrib/failmon/src/java/org/apache/hadoop/contrib/failmon/Environment.java
    hadoop/common/branches/branch-1-win/src/contrib/streaming/src/java/org/apache/hadoop/streaming/PathFinder.java
    hadoop/common/branches/branch-1-win/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
    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/fs/LocalFileSystem.java
    hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/io/nativeio/NativeIO.java
    hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/DiskChecker.java
    hadoop/common/branches/branch-1-win/src/hdfs/org/apache/hadoop/hdfs/server/common/Storage.java
    hadoop/common/branches/branch-1-win/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java
    hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/TaskLogServlet.java
    hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/pipes/Application.java
    hadoop/common/branches/branch-1-win/src/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
    hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/filecache/TestTrackerDistributedCacheManager.java
    hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/fs/TestFileUtil.java
    hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
    hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java
    hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/io/nativeio/TestNativeIO.java
    hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestJvmManager.java
    hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestKillSubProcesses.java
    hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestLinuxTaskControllerLaunchArgs.java
    hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestTaskLogsTruncater.java
    hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/util/TestShell.java
    hadoop/common/branches/branch-1-win/src/test/system/java/org/apache/hadoop/test/system/process/MultiUserHadoopDaemonRemoteCluster.java
    hadoop/common/branches/branch-1-win/src/test/testjar/GenerateTaskChildProcess.java
    hadoop/common/branches/branch-1-win/src/test/testshell/ExternalMapReduce.java
    hadoop/common/branches/branch-1-win/src/winutils/include/winutils.h
    hadoop/common/branches/branch-1-win/src/winutils/libwinutils.c

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=1512864&r1=1512863&r2=1512864&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 Sun Aug 11 03:17:21 2013
@@ -325,6 +325,9 @@ Branch-hadoop-1-win (branched from branc
 
     HADOOP-9790. Job token path is not unquoted properly. (Xi Fang via cnauroth)
 
+    HADOOP-9551. Backport common utils introduced with HADOOP-9413
+    to branch-1-win. (ivanmi)
+
   Merged from branch-1
 
     HDFS-385. Backport: Add support for an experimental API that allows a

Modified: hadoop/common/branches/branch-1-win/src/contrib/failmon/src/java/org/apache/hadoop/contrib/failmon/Environment.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/contrib/failmon/src/java/org/apache/hadoop/contrib/failmon/Environment.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/contrib/failmon/src/java/org/apache/hadoop/contrib/failmon/Environment.java (original)
+++ hadoop/common/branches/branch-1-win/src/contrib/failmon/src/java/org/apache/hadoop/contrib/failmon/Environment.java Sun Aug 11 03:17:21 2013
@@ -30,6 +30,7 @@ import java.util.regex.Pattern;
 
 import org.apache.commons.logging.*;
 import org.apache.log4j.PropertyConfigurator;
+import org.apache.hadoop.fs.FileUtil;
 
 /**********************************************************
  * This class provides various methods for interaction with
@@ -171,7 +172,7 @@ public class Environment {
     if ("true".equalsIgnoreCase(tmp) && fnames[0] != null)
       for (String fname : fnames) {
         File f = new File(fname);
-        if (f.exists() && f.canRead()) {
+        if (f.exists() && FileUtil.canRead(f)) {
           monitors.add(new MonitorJob(new HadoopLogParser(fname), "hadoopLog", timeInt));
 	  logInfo("Created Monitor for Hadoop log file: " + f.getAbsolutePath());
 	} else if (!f.exists())
@@ -192,7 +193,7 @@ public class Environment {
     if ("true".equalsIgnoreCase(tmp))
       for (String fname : fnames) {
         File f = new File(fname);
-        if (f.exists() && f.canRead()) {
+        if (f.exists() && FileUtil.canRead(f)) {
           monitors.add(new MonitorJob(new SystemLogParser(fname), "systemLog", timeInt));
 	  logInfo("Created Monitor for System log file: " + f.getAbsolutePath());
         } else if (!f.exists())

Modified: hadoop/common/branches/branch-1-win/src/contrib/streaming/src/java/org/apache/hadoop/streaming/PathFinder.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/contrib/streaming/src/java/org/apache/hadoop/streaming/PathFinder.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/contrib/streaming/src/java/org/apache/hadoop/streaming/PathFinder.java (original)
+++ hadoop/common/branches/branch-1-win/src/contrib/streaming/src/java/org/apache/hadoop/streaming/PathFinder.java Sun Aug 11 03:17:21 2013
@@ -22,6 +22,8 @@ import java.io.*;
 import java.util.*;
 import java.util.Map.Entry;
 
+import org.apache.hadoop.fs.FileUtil;
+
 /**
  * Maps a relative pathname to an absolute pathname using the
  * PATH enviroment.
@@ -94,7 +96,7 @@ public class PathFinder
         //
         // see if the filename matches and  we can read it
         //
-        if (f.isFile() && f.canRead()) {
+        if (f.isFile() && FileUtil.canRead(f)) {
           return f;
         }
       } catch (Exception exp){ }

Modified: hadoop/common/branches/branch-1-win/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java (original)
+++ hadoop/common/branches/branch-1-win/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java Sun Aug 11 03:17:21 2013
@@ -46,6 +46,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.filecache.DistributedCache;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
 import org.apache.hadoop.mapred.FileInputFormat;
@@ -353,7 +354,7 @@ public class StreamJob implements Tool {
   throws IllegalArgumentException {
     for (String file : values) {
       File f = new File(file);  
-      if (!f.canRead()) {
+      if (!FileUtil.canRead(f)) {
         fail("File: " + f.getAbsolutePath() 
           + " does not exist, or is not readable."); 
       }

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=1512864&r1=1512863&r2=1512864&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 Sun Aug 11 03:17:21 2013
@@ -742,6 +742,144 @@ public class FileUtil {
     execCommand(file, cmd);
   }
 
+
+  /**
+   * Platform independent implementation for {@link File#setReadable(boolean)}
+   * File#setReadable does not work as expected on Windows.
+   * @param f input file
+   * @param readable
+   * @return true on success, false otherwise
+   */
+  public static boolean setReadable(File f, boolean readable) {
+    if (Shell.WINDOWS) {
+      try {
+        String permission = readable ? "u+r" : "u-r";
+        FileUtil.chmod(f.getCanonicalPath(), permission, false);
+        return true;
+      } catch (IOException ex) {
+        return false;
+      }
+    } else {
+      return f.setReadable(readable);
+    }
+  }
+
+  /**
+   * Platform independent implementation for {@link File#setWritable(boolean)}
+   * File#setWritable does not work as expected on Windows.
+   * @param f input file
+   * @param writable
+   * @return true on success, false otherwise
+   */
+  public static boolean setWritable(File f, boolean writable) {
+    if (Shell.WINDOWS) {
+      try {
+        String permission = writable ? "u+w" : "u-w";
+        FileUtil.chmod(f.getCanonicalPath(), permission, false);
+        return true;
+      } catch (IOException ex) {
+        return false;
+      }
+    } else {
+      return f.setWritable(writable);
+    }
+  }
+
+  /**
+   * Platform independent implementation for {@link File#setExecutable(boolean)}
+   * File#setExecutable does not work as expected on Windows.
+   * Note: revoking execute permission on folders does not have the same
+   * behavior on Windows as on Unix platforms. Creating, deleting or renaming
+   * a file within that folder will still succeed on Windows.
+   * @param f input file
+   * @param executable
+   * @return true on success, false otherwise
+   */
+  public static boolean setExecutable(File f, boolean executable) {
+    if (Shell.WINDOWS) {
+      try {
+        String permission = executable ? "u+x" : "u-x";
+        FileUtil.chmod(f.getCanonicalPath(), permission, false);
+        return true;
+      } catch (IOException ex) {
+        return false;
+      }
+    } else {
+      return f.setExecutable(executable);
+    }
+  }
+
+  /**
+   * Simple wrapper function which checks whether native IO is available
+   * and warns if it is not.
+   */
+  private static boolean checkNativeAndWarn() {
+    boolean available = NativeIO.isAvailable();
+    if (!available) {
+      LOG.warn("NativeIO is not available, falling back to Java access"
+          + " check APIs for backward compatibility. Consider adding"
+          + " hadoop.dll to the java library path.");
+    }
+    return available;
+  }
+
+  /**
+   * Platform independent implementation for {@link File#canRead()}
+   * @param f input file
+   * @return On Unix, same as {@link File#canRead()}
+   *         On Windows, true if process has read access on the path
+   */
+  public static boolean canRead(File f) {
+    if (Shell.WINDOWS && checkNativeAndWarn()) {
+      try {
+        return NativeIO.Windows.access(f.getCanonicalPath(),
+            NativeIO.Windows.AccessRight.ACCESS_READ);
+      } catch (IOException e) {
+        return false;
+      }
+    } else {
+      return f.canRead();
+    }
+  }
+
+  /**
+   * Platform independent implementation for {@link File#canWrite()}
+   * @param f input file
+   * @return On Unix, same as {@link File#canWrite()}
+   *         On Windows, true if process has write access on the path
+   */
+  public static boolean canWrite(File f) {
+    if (Shell.WINDOWS && checkNativeAndWarn()) {
+      try {
+        return NativeIO.Windows.access(f.getCanonicalPath(),
+            NativeIO.Windows.AccessRight.ACCESS_WRITE);
+      } catch (IOException e) {
+        return false;
+      }
+    } else {
+      return f.canWrite();
+    }
+  }
+
+  /**
+   * Platform independent implementation for {@link File#canExecute()}
+   * @param f input file
+   * @return On Unix, same as {@link File#canExecute()}
+   *         On Windows, true if process has execute access on the path
+   */
+  public static boolean canExecute(File f) {
+    if (Shell.WINDOWS && checkNativeAndWarn()) {
+      try {
+        return NativeIO.Windows.access(f.getCanonicalPath(),
+            NativeIO.Windows.AccessRight.ACCESS_EXECUTE);
+      } catch (IOException e) {
+        return false;
+      }
+    } else {
+      return f.canExecute();
+    }
+  }
+
   /**
    * Set permissions to the required value. Uses the java primitives instead
    * of forking if group == other.

Modified: hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/fs/LocalFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/fs/LocalFileSystem.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/fs/LocalFileSystem.java (original)
+++ hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/fs/LocalFileSystem.java Sun Aug 11 03:17:21 2013
@@ -88,7 +88,7 @@ public class LocalFileSystem extends Che
       String device = new DF(f, getConf()).getMount();
       File parent = f.getParentFile();
       File dir = null;
-      while (parent!=null && parent.canWrite() && parent.toString().startsWith(device)) {
+      while (parent!=null && FileUtil.canWrite(parent) && parent.toString().startsWith(device)) {
         dir = parent;
         parent = parent.getParentFile();
       }

Modified: hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/io/nativeio/NativeIO.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/io/nativeio/NativeIO.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/io/nativeio/NativeIO.java (original)
+++ hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/io/nativeio/NativeIO.java Sun Aug 11 03:17:21 2013
@@ -177,6 +177,43 @@ public class NativeIO {
     /** Windows only methods used for getOwner() implementation */
     private static native String getOwner(FileDescriptor fd) throws IOException;
 
+    /** Supported list of Windows access right flags */
+    public static enum AccessRight {
+      ACCESS_READ (0x0001),      // FILE_READ_DATA
+      ACCESS_WRITE (0x0002),     // FILE_WRITE_DATA
+      ACCESS_EXECUTE (0x0020);   // FILE_EXECUTE
+
+      private final int accessRight;
+      AccessRight(int access) {
+        accessRight = access;
+      }
+
+      public int accessRight() {
+        return accessRight;
+      }
+    };
+
+    /** Windows only method used to check if the current process has requested
+     *  access rights on the given path. */
+    private static native boolean access0(String path, int requestedAccess);
+
+    /**
+     * Checks whether the current process has desired access rights on
+     * the given path.
+     * 
+     * Longer term this native function can be substituted with JDK7
+     * function Files#isReadable, isWritable, isExecutable.
+     *
+     * @param path input path
+     * @param desiredAccess ACCESS_READ, ACCESS_WRITE or ACCESS_EXECUTE
+     * @return true if access is allowed
+     * @throws IOException I/O exception on error
+     */
+    public static boolean access(String path, AccessRight desiredAccess)
+        throws IOException {
+      return access0(path, desiredAccess.accessRight());
+    }
+
     static {
       if (NativeCodeLoader.isNativeCodeLoaded()) {
         try {

Modified: hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/DiskChecker.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/DiskChecker.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/DiskChecker.java (original)
+++ hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/DiskChecker.java Sun Aug 11 03:17:21 2013
@@ -21,6 +21,7 @@ package org.apache.hadoop.util;
 import java.io.File;
 import java.io.IOException;
 
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -165,11 +166,7 @@ public class DiskChecker {
                                    + dir.toString());
     }
 
-    if (Shell.WINDOWS) {
-      checkAccessByFileSystemInteraction(dir);
-    } else {
-      checkAccessByFileMethods(dir);
-    }
+    checkAccessByFileMethods(dir);
   }
 
   /**
@@ -182,68 +179,19 @@ public class DiskChecker {
    */
   private static void checkAccessByFileMethods(File dir)
       throws DiskErrorException {
-    if (!dir.canRead()) {
+    if (!FileUtil.canRead(dir)) {
       throw new DiskErrorException("Directory is not readable: "
                                    + dir.toString());
     }
 
-    if (!dir.canWrite()) {
+    if (!FileUtil.canWrite(dir)) {
       throw new DiskErrorException("Directory is not writable: "
                                    + dir.toString());
     }
 
-    if (!dir.canExecute()) {
+    if (!FileUtil.canExecute(dir)) {
       throw new DiskErrorException("Directory is not executable: "
                                    + dir.toString());
     }
   }
-
-  /**
-   * Checks that the current running process can read, write, and execute the
-   * given directory by attempting each of those operations on the file system.
-   * This method contains several workarounds to known JVM bugs that cause
-   * File.canRead, File.canWrite, and File.canExecute to return incorrect results
-   * on Windows with NTFS ACLs.  See:
-   * http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6203387
-   * These bugs are supposed to be fixed in JDK7.
-   * 
-   * @param dir File to check
-   * @throws DiskErrorException if dir is not readable, not writable, or not
-   *   executable
-   */
-  private static void checkAccessByFileSystemInteraction(File dir)
-      throws DiskErrorException {
-    // Make sure we can read the directory by listing it.
-    if (dir.list() == null) {
-      throw new DiskErrorException("Directory is not readable: "
-                                   + dir.toString());
-    }
-
-    // Make sure we can write to the directory by creating a temp file in it.
-    try {
-      File tempFile = File.createTempFile("checkDirAccess", null, dir);
-      if (!tempFile.delete()) {
-        throw new DiskErrorException("Directory is not writable: "
-                                     + dir.toString());
-      }
-    } catch (IOException e) {
-      throw new DiskErrorException("Directory is not writable: "
-                                   + dir.toString(), e);
-    }
-
-    // Make sure the directory is executable by trying to cd into it.  This
-    // launches a separate process.  It does not change the working directory of
-    // the current process.
-    try {
-      String[] cdCmd = new String[] { "cmd", "/C", "cd",
-          dir.getAbsolutePath() };
-      Shell.execCommand(null, cdCmd, SHELL_TIMEOUT);
-    } catch (Shell.ExitCodeException e) {
-      throw new DiskErrorException("Directory is not executable: "
-                                   + dir.toString(), e);
-    } catch (IOException e) {
-      throw new DiskErrorException("Directory is not executable: "
-                                   + dir.toString(), e);
-    }
-  }
 }

Modified: hadoop/common/branches/branch-1-win/src/hdfs/org/apache/hadoop/hdfs/server/common/Storage.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/hdfs/org/apache/hadoop/hdfs/server/common/Storage.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/hdfs/org/apache/hadoop/hdfs/server/common/Storage.java (original)
+++ hadoop/common/branches/branch-1-win/src/hdfs/org/apache/hadoop/hdfs/server/common/Storage.java Sun Aug 11 03:17:21 2013
@@ -440,7 +440,7 @@ public abstract class Storage extends St
           LOG.info(rootPath + "is not a directory");
           return StorageState.NON_EXISTENT;
         }
-        if (!root.canWrite()) {
+        if (!FileUtil.canWrite(root)) {
           LOG.info("Cannot access storage directory " + rootPath);
           return StorageState.NON_EXISTENT;
         }

Modified: hadoop/common/branches/branch-1-win/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java (original)
+++ hadoop/common/branches/branch-1-win/src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSImage.java Sun Aug 11 03:17:21 2013
@@ -565,7 +565,7 @@ public class FSImage extends Storage {
   long readCheckpointTime(StorageDirectory sd) throws IOException {
     File timeFile = getImageFile(sd, NameNodeFile.TIME);
     long timeStamp = 0L;
-    if (timeFile.exists() && timeFile.canRead()) {
+    if (timeFile.exists() && FileUtil.canRead(timeFile)) {
       DataInputStream in = new DataInputStream(new FileInputStream(timeFile));
       try {
         timeStamp = in.readLong();

Modified: hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/TaskLogServlet.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/TaskLogServlet.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/TaskLogServlet.java (original)
+++ hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/TaskLogServlet.java Sun Aug 11 03:17:21 2013
@@ -30,6 +30,7 @@ import javax.servlet.http.HttpServletRes
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.QueueManager.QueueACL;
 import org.apache.hadoop.mapreduce.JobACL;
@@ -51,7 +52,7 @@ public class TaskLogServlet extends Http
   private boolean haveTaskLog(TaskAttemptID taskId, boolean isCleanup,
       TaskLog.LogName type) {
     File f = TaskLog.getTaskLogFile(taskId, isCleanup, type);
-    return f.canRead();
+    return FileUtil.canRead(f);
   }
 
   /**

Modified: hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/pipes/Application.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/pipes/Application.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/pipes/Application.java (original)
+++ hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/pipes/Application.java Sun Aug 11 03:17:21 2013
@@ -113,7 +113,7 @@ class Application<K1 extends WritableCom
     }
 
     String executable = DistributedCache.getLocalCacheFiles(conf)[0].toString();
-    if (!new File(executable).canExecute()) {
+    if (!FileUtil.canExecute(new File(executable))) {
       // LinuxTaskController sets +x permissions on all distcache files already.
       // In case of DefaultTaskController, set permissions here.
       FileUtil.chmod(executable, "u+x");

Modified: hadoop/common/branches/branch-1-win/src/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/native/src/org/apache/hadoop/io/nativeio/NativeIO.c?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/native/src/org/apache/hadoop/io/nativeio/NativeIO.c (original)
+++ hadoop/common/branches/branch-1-win/src/native/src/org/apache/hadoop/io/nativeio/NativeIO.c Sun Aug 11 03:17:21 2013
@@ -679,6 +679,42 @@ cleanup:
 #endif
 }
 
+/*
+ * Class:     org_apache_hadoop_io_nativeio_NativeIO_Windows
+ * Method:    access0
+ * Signature: (Ljava/lang/String;I)Z
+ */
+JNIEXPORT jboolean JNICALL Java_org_apache_hadoop_io_nativeio_NativeIO_00024Windows_access0
+  (JNIEnv *env, jclass clazz, jstring jpath, jint jaccess)
+{
+#ifdef UNIX
+  THROW(env, "java/io/IOException",
+    "The function access0(path, access) is not supported on Unix");
+  return NULL;
+#endif
+
+#ifdef WINDOWS
+  LPCWSTR path = NULL;
+  DWORD dwRtnCode = ERROR_SUCCESS;
+  ACCESS_MASK access = (ACCESS_MASK)jaccess;
+  BOOL allowed = FALSE;
+
+  path = (LPCWSTR) (*env)->GetStringChars(env, jpath, NULL);
+  if (!path) goto cleanup; // exception was thrown
+
+  dwRtnCode = CheckAccessForCurrentUser(path, access, &allowed);
+  if (dwRtnCode != ERROR_SUCCESS) {
+    throw_ioe(env, dwRtnCode);
+    goto cleanup;
+  }
+
+cleanup:
+  if (path) (*env)->ReleaseStringChars(env, jpath, path);
+
+  return (jboolean)allowed;
+#endif
+}
+
 /**
  * vim: sw=2: ts=2: et:
  */

Modified: hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/filecache/TestTrackerDistributedCacheManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/filecache/TestTrackerDistributedCacheManager.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/filecache/TestTrackerDistributedCacheManager.java (original)
+++ hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/filecache/TestTrackerDistributedCacheManager.java Sun Aug 11 03:17:21 2013
@@ -552,8 +552,8 @@ public class TestTrackerDistributedCache
       throws IOException {
     // All the files should have executable permissions on them.
     for (Path p : localCacheFiles) {
-      assertTrue("Cache file is not executable!", new File(p
-          .toUri().getPath()).canExecute());
+      assertTrue("Cache file is not executable!", FileUtil.canExecute(
+          new File(p.toUri().getPath())));
     }
   }
 

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=1512864&r1=1512863&r2=1512864&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 Sun Aug 11 03:17:21 2013
@@ -155,7 +155,7 @@ public class TestFileUtil {
 
     xSubDir.mkdirs();
     file2.createNewFile();
-    xSubDir.setWritable(false);
+    FileUtil.setWritable(xSubDir, false);
     ySubDir.mkdirs();
     file3.createNewFile();
 
@@ -172,7 +172,7 @@ public class TestFileUtil {
   // Sets writable permissions for the non-deleted dir "xsubdir" so that it can
   // be deleted in tearDown().
   private void validateAndSetWritablePermissions(boolean ret) {
-    xSubDir.setWritable(true);
+    FileUtil.setWritable(xSubDir, true);
     Assert.assertFalse("The return value should have been false!", ret);
     Assert.assertTrue("The file file1 should not have been deleted!",
         new File(del, file1Name).exists());

Modified: hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java (original)
+++ hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java Sun Aug 11 03:17:21 2013
@@ -356,10 +356,10 @@ public class TestDataNodeVolumeFailure e
   //@After
   public void tearDown() throws Exception {
     if(data_fail != null) {
-      data_fail.setWritable(true);
+      FileUtil.setWritable(data_fail, true);
     }
     if(failedDir != null) {
-      failedDir.setWritable(true);
+      FileUtil.setWritable(failedDir, true);
     }
     if(cluster != null) {
       cluster.shutdown();

Modified: hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java (original)
+++ hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java Sun Aug 11 03:17:21 2013
@@ -24,6 +24,7 @@ import java.net.Socket;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -75,8 +76,8 @@ public class TestDiskError extends TestC
       }
     } finally {
       // restore its old permission
-      dir1.setWritable(true);
-      dir2.setWritable(true);
+      FileUtil.setWritable(dir1, true);
+      FileUtil.setWritable(dir2, true);
       cluster.shutdown();
     }
   }

Modified: hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/io/nativeio/TestNativeIO.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/io/nativeio/TestNativeIO.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/io/nativeio/TestNativeIO.java (original)
+++ hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/io/nativeio/TestNativeIO.java Sun Aug 11 03:17:21 2013
@@ -192,6 +192,44 @@ public class TestNativeIO {
 
   }
 
+  /** Validate access checks on Windows */
+  @Test
+  public void testAccess() throws Exception {
+    if (!Shell.WINDOWS) {
+      return;
+    }
+
+    File testFile = new File(TEST_DIR, "testfileaccess");
+    assertTrue(testFile.createNewFile());
+
+    // Validate ACCESS_READ
+    FileUtil.setReadable(testFile, false);
+    assertFalse(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_READ));
+
+    FileUtil.setReadable(testFile, true);
+    assertTrue(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_READ));
+
+    // Validate ACCESS_WRITE
+    FileUtil.setWritable(testFile, false);
+    assertFalse(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_WRITE));
+
+    FileUtil.setWritable(testFile, true);
+    assertTrue(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_WRITE));
+
+    // Validate ACCESS_EXECUTE
+    FileUtil.setExecutable(testFile, false);
+    assertFalse(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_EXECUTE));
+
+    FileUtil.setExecutable(testFile, true);
+    assertTrue(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_EXECUTE));
+  }
+
   @Test
   public void testOpen() throws Exception {
     if (Shell.WINDOWS)

Modified: hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestJvmManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestJvmManager.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestJvmManager.java (original)
+++ hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestJvmManager.java Sun Aug 11 03:17:21 2013
@@ -111,7 +111,7 @@ public class TestJvmManager {
     // write the actual command it self.
     out.write(cmd.getBytes());
     out.close();
-    script.setExecutable(true);
+    FileUtil.setExecutable(script, true);
     return script;
   }
   
@@ -119,7 +119,7 @@ public class TestJvmManager {
   private File writeEmptyScript(String fileName) throws IOException {
     File script = new File(TEST_DIR, fileName);
     script.createNewFile();
-    script.setExecutable(true);
+    FileUtil.setExecutable(script, true);
     return script;
   }
   

Modified: hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestKillSubProcesses.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestKillSubProcesses.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestKillSubProcesses.java (original)
+++ hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestKillSubProcesses.java Sun Aug 11 03:17:21 2013
@@ -28,6 +28,7 @@ import java.util.StringTokenizer;
 import junit.framework.TestCase;
 
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -452,7 +453,7 @@ public class TestKillSubProcesses extend
       file.close();
 
       // Set executable permissions on the script.
-      new File(scriptPath.toUri().getPath()).setExecutable(true);
+      FileUtil.setExecutable(new File(scriptPath.toUri().getPath()), true);
 
       LOG.info("Calling script from map task : " + shellScript);
       

Modified: hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestLinuxTaskControllerLaunchArgs.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestLinuxTaskControllerLaunchArgs.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestLinuxTaskControllerLaunchArgs.java (original)
+++ hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestLinuxTaskControllerLaunchArgs.java Sun Aug 11 03:17:21 2013
@@ -88,7 +88,7 @@ public class TestLinuxTaskControllerLaun
     out.write("LTC_ARGS=$((LTC_ARG1+LTC_ARG2+LTC_ARG3+LTC_ARG4+LTC_ARG5))\n");
     out.write("if [ $LTC_ARGS -eq 5 ]; then exit 0; else exit 22; fi\n");
     out.close();
-    fakeTaskController.setExecutable(true);
+    FileUtil.setExecutable(fakeTaskController, true);
   }
 
   protected void initMyTest() throws Exception {

Modified: hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestTaskLogsTruncater.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestTaskLogsTruncater.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestTaskLogsTruncater.java (original)
+++ hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestTaskLogsTruncater.java Sun Aug 11 03:17:21 2013
@@ -70,7 +70,7 @@ public class TestTaskLogsTruncater {
   public void tearDown() throws IOException {
     File logDir = TaskLog.getUserLogDir();
     for (File attemptDir : logDir.listFiles()) {
-      attemptDir.setWritable(true);
+      FileUtil.setWritable(attemptDir, true);
       FileUtil.fullyDelete(attemptDir);
     }
   }
@@ -605,7 +605,7 @@ public class TestTaskLogsTruncater {
       DataOutputStream scriptFile = fs.create(scriptPath);
       scriptFile.writeBytes(debugScriptContent);
       scriptFile.close();
-      new File(scriptPath.toUri().getPath()).setExecutable(true);
+      FileUtil.setExecutable(new File(scriptPath.toUri().getPath()), true);
 
       URI uri = scriptPath.toUri();
       DistributedCache.createSymlink(conf);

Modified: hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/util/TestShell.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/util/TestShell.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/util/TestShell.java (original)
+++ hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/util/TestShell.java Sun Aug 11 03:17:21 2013
@@ -25,6 +25,8 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.PrintWriter;
 
+import org.apache.hadoop.fs.FileUtil;
+
 public class TestShell extends TestCase {
 
   private static class Command extends Shell {
@@ -90,7 +92,7 @@ public class TestShell extends TestCase 
     PrintWriter writer = new PrintWriter(new FileOutputStream(shellFile));
     writer.println(timeoutCommand);
     writer.close();
-    shellFile.setExecutable(true);
+    FileUtil.setExecutable(shellFile, true);
     Shell.ShellCommandExecutor shexc 
     = new Shell.ShellCommandExecutor(new String[]{shellFile.getAbsolutePath()},
                                       null, null, 100);

Modified: hadoop/common/branches/branch-1-win/src/test/system/java/org/apache/hadoop/test/system/process/MultiUserHadoopDaemonRemoteCluster.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/test/system/java/org/apache/hadoop/test/system/process/MultiUserHadoopDaemonRemoteCluster.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/test/system/java/org/apache/hadoop/test/system/process/MultiUserHadoopDaemonRemoteCluster.java (original)
+++ hadoop/common/branches/branch-1-win/src/test/system/java/org/apache/hadoop/test/system/process/MultiUserHadoopDaemonRemoteCluster.java Sun Aug 11 03:17:21 2013
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.test.system.process.HadoopDaemonRemoteCluster.HadoopDaemonInfo;
 
 public abstract class MultiUserHadoopDaemonRemoteCluster
@@ -68,7 +69,7 @@ public abstract class MultiUserHadoopDae
                 + MULTI_USER_BINARY_PATH_KEY + " correctly");
       }
       File binaryFile = new File(binaryPath);
-      if (!binaryFile.exists() || !binaryFile.canExecute()) {
+      if (!binaryFile.exists() || !FileUtil.canExecute(binaryFile)) {
         throw new IllegalArgumentException(
             "Binary file path is not configured correctly. Please set "
                 + MULTI_USER_BINARY_PATH_KEY

Modified: hadoop/common/branches/branch-1-win/src/test/testjar/GenerateTaskChildProcess.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/test/testjar/GenerateTaskChildProcess.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/test/testjar/GenerateTaskChildProcess.java (original)
+++ hadoop/common/branches/branch-1-win/src/test/testjar/GenerateTaskChildProcess.java Sun Aug 11 03:17:21 2013
@@ -29,6 +29,7 @@ import org.apache.hadoop.mapred.MapReduc
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -248,7 +249,7 @@ public class GenerateTaskChildProcess {
     file.writeBytes(script);
     file.close();
     File scriptFile = new File(scriptDirName,scriptName);
-    scriptFile.setExecutable(true);
+    FileUtil.setExecutable(scriptFile, true);
     LOG.info("script absolute path:" + scriptFile.getAbsolutePath());
     String [] cmd = new String[]{scriptFile.getAbsolutePath(), 
             String.valueOf(numOfChildProcesses)};

Modified: hadoop/common/branches/branch-1-win/src/test/testshell/ExternalMapReduce.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/test/testshell/ExternalMapReduce.java?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/test/testshell/ExternalMapReduce.java (original)
+++ hadoop/common/branches/branch-1-win/src/test/testshell/ExternalMapReduce.java Sun Aug 11 03:17:21 2013
@@ -29,6 +29,7 @@ import java.util.jar.Manifest;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Writable;
@@ -167,7 +168,7 @@ public class ExternalMapReduce extends C
       verifyFilesExist(expectedFileList);
 
       File file = new File("./jarlink/test.txt");
-      if (!file.canExecute()) {
+      if (!FileUtil.canExecute(file)) {
         throw new IOException("jarlink/test.txt is not executable");
       }
     }

Modified: hadoop/common/branches/branch-1-win/src/winutils/include/winutils.h
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/winutils/include/winutils.h?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/winutils/include/winutils.h (original)
+++ hadoop/common/branches/branch-1-win/src/winutils/include/winutils.h Sun Aug 11 03:17:21 2013
@@ -110,6 +110,11 @@ void SystemInfoUsage();
 DWORD GetFileInformationByName(__in LPCWSTR pathName,  __in BOOL followLink,
   __out LPBY_HANDLE_FILE_INFORMATION lpFileInformation);
 
+DWORD CheckAccessForCurrentUser(
+  __in PCWSTR pathName,
+  __in ACCESS_MASK requestedAccess,
+  __out BOOL *allowed);
+
 DWORD ConvertToLongPath(__in PCWSTR path, __deref_out PWSTR *newPath);
 
 DWORD GetSidFromAcctNameW(__in PCWSTR acctName, __out PSID* ppSid);

Modified: hadoop/common/branches/branch-1-win/src/winutils/libwinutils.c
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/winutils/libwinutils.c?rev=1512864&r1=1512863&r2=1512864&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/winutils/libwinutils.c (original)
+++ hadoop/common/branches/branch-1-win/src/winutils/libwinutils.c Sun Aug 11 03:17:21 2013
@@ -567,7 +567,7 @@ static DWORD GetEffectiveRightsForSid(PS
   PSID pSid,
   PACCESS_MASK pAccessRights)
 {
-  AUTHZ_RESOURCE_MANAGER_HANDLE hManager;
+  AUTHZ_RESOURCE_MANAGER_HANDLE hManager = NULL;
   LUID unusedId = { 0 };
   AUTHZ_CLIENT_CONTEXT_HANDLE hAuthzClientContext = NULL;
   DWORD dwRtnCode = ERROR_SUCCESS;
@@ -581,6 +581,10 @@ static DWORD GetEffectiveRightsForSid(PS
     return GetLastError();
   }
 
+  // Pass AUTHZ_SKIP_TOKEN_GROUPS to the function to avoid querying user group
+  // information for access check. This allows us to model POSIX permissions
+  // on Windows, where a user can have less permissions than a group it
+  // belongs to.
   if(!AuthzInitializeContextFromSid(AUTHZ_SKIP_TOKEN_GROUPS,
     pSid, hManager, NULL, unusedId, NULL, &hAuthzClientContext))
   {
@@ -594,17 +598,116 @@ static DWORD GetEffectiveRightsForSid(PS
     ret = dwRtnCode;
     goto GetEffectiveRightsForSidEnd;
   }
-  if (!AuthzFreeContext(hAuthzClientContext))
+
+GetEffectiveRightsForSidEnd:
+  if (hManager != NULL)
   {
-    ret = GetLastError();
-    goto GetEffectiveRightsForSidEnd;
+    (void)AuthzFreeResourceManager(hManager);
+  }
+  if (hAuthzClientContext != NULL)
+  {
+    (void)AuthzFreeContext(hAuthzClientContext);
   }
 
-GetEffectiveRightsForSidEnd:
   return ret;
 }
 
 //----------------------------------------------------------------------------
+// Function: CheckAccessForCurrentUser
+//
+// Description:
+//   Checks if the current process has the requested access rights on the given
+//   path. Based on the following MSDN article:
+//   http://msdn.microsoft.com/en-us/library/windows/desktop/ff394771(v=vs.85).aspx
+//
+// Returns:
+//   ERROR_SUCCESS: on success
+//
+DWORD CheckAccessForCurrentUser(
+  __in PCWSTR pathName,
+  __in ACCESS_MASK requestedAccess,
+  __out BOOL *allowed)
+{
+  DWORD dwRtnCode = ERROR_SUCCESS;
+
+  LPWSTR longPathName = NULL;
+  HANDLE hProcessToken = NULL;
+  PSECURITY_DESCRIPTOR pSd = NULL;
+
+  AUTHZ_RESOURCE_MANAGER_HANDLE hManager = NULL;
+  AUTHZ_CLIENT_CONTEXT_HANDLE hAuthzClientContext = NULL;
+  LUID Luid = {0, 0};
+
+  ACCESS_MASK currentUserAccessRights = 0;
+
+  // Prepend the long path prefix if needed
+  dwRtnCode = ConvertToLongPath(pathName, &longPathName);
+  if (dwRtnCode != ERROR_SUCCESS)
+  {
+    goto CheckAccessEnd;
+  }
+
+  // Get SD of the given path. OWNER and DACL security info must be
+  // requested, otherwise, AuthzAccessCheck fails with invalid parameter
+  // error.
+  dwRtnCode = GetNamedSecurityInfo(longPathName, SE_FILE_OBJECT,
+    OWNER_SECURITY_INFORMATION | GROUP_SECURITY_INFORMATION |
+    DACL_SECURITY_INFORMATION,
+    NULL, NULL, NULL, NULL, &pSd);
+  if (dwRtnCode != ERROR_SUCCESS)
+  {
+    goto CheckAccessEnd;
+  }
+
+  // Get current process token
+  if (!OpenProcessToken(GetCurrentProcess(), TOKEN_QUERY, &hProcessToken))
+  {
+    dwRtnCode = GetLastError();
+    goto CheckAccessEnd;
+  }
+
+  if (!AuthzInitializeResourceManager(AUTHZ_RM_FLAG_NO_AUDIT, NULL, NULL,
+    NULL, NULL, &hManager))
+  {
+    dwRtnCode = GetLastError();
+    goto CheckAccessEnd;
+  }
+
+  if(!AuthzInitializeContextFromToken(0, hProcessToken, hManager, NULL,
+    Luid, NULL, &hAuthzClientContext))
+  {
+    dwRtnCode = GetLastError();
+    goto CheckAccessEnd;
+  }
+
+  dwRtnCode = GetAccess(hAuthzClientContext, pSd, &currentUserAccessRights);
+  if (dwRtnCode != ERROR_SUCCESS)
+  {
+    goto CheckAccessEnd;
+  }
+
+  *allowed = ((currentUserAccessRights & requestedAccess) == requestedAccess);
+
+CheckAccessEnd:
+  LocalFree(longPathName);
+  LocalFree(pSd);
+  if (hProcessToken != NULL)
+  {
+    CloseHandle(hProcessToken);
+  }
+  if (hManager != NULL)
+  {
+    (void)AuthzFreeResourceManager(hManager);
+  }
+  if (hAuthzClientContext != NULL)
+  {
+    (void)AuthzFreeContext(hAuthzClientContext);
+  }
+
+  return dwRtnCode;
+}
+
+//----------------------------------------------------------------------------
 // Function: FindFileOwnerAndPermission
 //
 // Description: