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 ji...@apache.org on 2014/10/23 01:26:58 UTC

[4/4] git commit: YARN-2198. Remove the need to run NodeManager as privileged account for Windows Secure Container Executor. Contributed by Remus Rusanu (cherry picked from commit 3b12fd6cfbf4cc91ef8e8616c7aafa9de006cde5)

YARN-2198. Remove the need to run NodeManager as privileged account for Windows Secure Container Executor. Contributed by Remus Rusanu
(cherry picked from commit 3b12fd6cfbf4cc91ef8e8616c7aafa9de006cde5)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1c235a44
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1c235a44
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1c235a44

Branch: refs/heads/branch-2
Commit: 1c235a44489deac9792ee3131f3dd334c5d198ab
Parents: c8d0758
Author: Jian He <ji...@apache.org>
Authored: Wed Oct 22 15:57:46 2014 -0700
Committer: Jian He <ji...@apache.org>
Committed: Wed Oct 22 15:58:26 2014 -0700

----------------------------------------------------------------------
 .gitignore                                      |    3 +
 hadoop-common-project/hadoop-common/pom.xml     |    5 +
 .../java/org/apache/hadoop/fs/FileUtil.java     |    9 +-
 .../apache/hadoop/fs/RawLocalFileSystem.java    |   18 +-
 .../org/apache/hadoop/io/nativeio/NativeIO.java |    5 +
 .../main/java/org/apache/hadoop/util/Shell.java |   19 +-
 .../src/main/native/native.vcxproj              |    4 +
 .../org/apache/hadoop/io/nativeio/NativeIO.c    |   11 +-
 .../windows_secure_container_executor.c         |  569 +++++++
 .../windows_secure_container_executor.h         |   27 +
 .../hadoop-common/src/main/winutils/chown.c     |  130 +-
 .../hadoop-common/src/main/winutils/client.c    |  498 ++++++
 .../hadoop-common/src/main/winutils/config.cpp  |  174 ++
 .../src/main/winutils/hadoopwinutilsvc.idl      |  130 ++
 .../src/main/winutils/include/winutils.h        |  114 +-
 .../src/main/winutils/libwinutils.c             |  748 ++++++++-
 .../src/main/winutils/libwinutils.vcxproj       |   90 +-
 .../hadoop-common/src/main/winutils/main.c      |   23 +
 .../hadoop-common/src/main/winutils/service.c   | 1485 ++++++++++++++++++
 .../hadoop-common/src/main/winutils/task.c      | 1207 ++++++++++----
 .../hadoop-common/src/main/winutils/winutils.mc |   64 +
 .../src/main/winutils/winutils.sln              |   18 +-
 .../src/main/winutils/winutils.vcxproj          |  106 +-
 .../hadoop/mapreduce/util/ProcessTree.java      |    4 +-
 hadoop-yarn-project/CHANGES.txt                 |    3 +
 .../server/nodemanager/ContainerExecutor.java   |   25 +-
 .../nodemanager/DefaultContainerExecutor.java   |   73 +-
 .../nodemanager/LinuxContainerExecutor.java     |    5 +-
 .../WindowsSecureContainerExecutor.java         |  653 +++++++-
 .../launcher/ContainerLaunch.java               |   20 +-
 .../localizer/ContainerLocalizer.java           |    7 +-
 .../localizer/ResourceLocalizationService.java  |    3 +-
 .../TestDefaultContainerExecutor.java           |    8 +-
 .../nodemanager/TestLinuxContainerExecutor.java |    4 +-
 .../TestLinuxContainerExecutorWithMocks.java    |    2 +-
 .../TestResourceLocalizationService.java        |    2 +-
 .../src/site/apt/SecureContainer.apt.vm         |   68 +-
 37 files changed, 5551 insertions(+), 783 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 9d81253..8b132cb 100644
--- a/.gitignore
+++ b/.gitignore
@@ -4,6 +4,9 @@
 *.orig
 *.rej
 **/.keep
+*.sdf
+*.suo
+*.vcxproj.user
 .idea
 .svn
 .classpath

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml
index 6cdced2..3e7bf0c 100644
--- a/hadoop-common-project/hadoop-common/pom.xml
+++ b/hadoop-common-project/hadoop-common/pom.xml
@@ -33,6 +33,8 @@
   <properties>
     <hadoop.component>common</hadoop.component>
     <is.hadoop.component>true</is.hadoop.component>
+    <wsce.config.dir>../etc/hadoop</wsce.config.dir>
+    <wsce.config.file>wsce-site.xml</wsce.config.file>
   </properties>
 
   <dependencies>
@@ -735,6 +737,9 @@
                     <argument>/nologo</argument>
                     <argument>/p:Configuration=Release</argument>
                     <argument>/p:OutDir=${project.build.directory}/bin/</argument>
+                    <argument>/p:IntermediateOutputPath=${project.build.directory}/winutils/</argument>
+                    <argument>/p:WsceConfigDir=${wsce.config.dir}</argument>
+                    <argument>/p:WsceConfigFile=${wsce.config.file}</argument>
                   </arguments>
                 </configuration>
               </execution>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
index 0848cb1..91f00e1 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
@@ -1186,6 +1186,11 @@ public class FileUtil {
     return fileNames;
   }  
   
+  public static String[] createJarWithClassPath(String inputClassPath, Path pwd,
+      Map<String, String> callerEnv) throws IOException {
+    return createJarWithClassPath(inputClassPath, pwd, pwd, callerEnv);
+  }
+  
   /**
    * Create a jar file at the given path, containing a manifest with a classpath
    * that references all specified entries.
@@ -1210,6 +1215,7 @@ public class FileUtil {
    * 
    * @param inputClassPath String input classpath to bundle into the jar manifest
    * @param pwd Path to working directory to save jar
+   * @param targetDir path to where the jar execution will have its working dir
    * @param callerEnv Map<String, String> caller's environment variables to use
    *   for expansion
    * @return String[] with absolute path to new jar in position 0 and
@@ -1217,6 +1223,7 @@ public class FileUtil {
    * @throws IOException if there is an I/O error while writing the jar file
    */
   public static String[] createJarWithClassPath(String inputClassPath, Path pwd,
+      Path targetDir,
       Map<String, String> callerEnv) throws IOException {
     // Replace environment variables, case-insensitive on Windows
     @SuppressWarnings("unchecked")
@@ -1265,7 +1272,7 @@ public class FileUtil {
         // Append just this entry
         File fileCpEntry = null;
         if(!new Path(classPathEntry).isAbsolute()) {
-          fileCpEntry = new File(workingDir, classPathEntry);
+          fileCpEntry = new File(targetDir.toString(), classPathEntry);
         }
         else {
           fileCpEntry = new File(classPathEntry);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
index a9e02da..ca815a3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
@@ -268,7 +268,12 @@ public class RawLocalFileSystem extends FileSystem {
       throw new IOException("Mkdirs failed to create " + parent.toString());
     }
     return new FSDataOutputStream(new BufferedOutputStream(
-        new LocalFSFileOutputStream(f, false), bufferSize), statistics);
+        createOutputStream(f, false), bufferSize), statistics);
+  }
+  
+  protected OutputStream createOutputStream(Path f, boolean append) 
+      throws IOException {
+    return new LocalFSFileOutputStream(f, append); 
   }
   
   @Override
@@ -400,6 +405,10 @@ public class RawLocalFileSystem extends FileSystem {
     }
     return Arrays.copyOf(results, j);
   }
+  
+  protected boolean mkOneDir(File p2f) throws IOException {
+    return p2f.mkdir();
+  }
 
   /**
    * Creates the specified directory hierarchy. Does not
@@ -412,8 +421,9 @@ public class RawLocalFileSystem extends FileSystem {
     }
     Path parent = f.getParent();
     File p2f = pathToFile(f);
+    File parent2f = null;
     if(parent != null) {
-      File parent2f = pathToFile(parent);
+      parent2f = pathToFile(parent);
       if(parent2f != null && parent2f.exists() && !parent2f.isDirectory()) {
         throw new ParentNotDirectoryException("Parent path is not a directory: "
             + parent);
@@ -423,8 +433,8 @@ public class RawLocalFileSystem extends FileSystem {
       throw new FileNotFoundException("Destination exists" +
               " and is not a directory: " + p2f.getCanonicalPath());
     }
-    return (parent == null || mkdirs(parent)) &&
-      (p2f.mkdir() || p2f.isDirectory());
+    return (parent == null || parent2f.exists() || mkdirs(parent)) &&
+      (mkOneDir(p2f) || p2f.isDirectory());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
index 43f1cb1..2400958 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
@@ -22,6 +22,8 @@ import java.io.FileDescriptor;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.io.RandomAccessFile;
 import java.lang.reflect.Field;
 import java.nio.ByteBuffer;
@@ -35,6 +37,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.HardLink;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.SecureIOUtils.AlreadyExistsException;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.apache.hadoop.util.Shell;
@@ -504,6 +507,8 @@ public class NativeIO {
     public static final long FILE_BEGIN = 0;
     public static final long FILE_CURRENT = 1;
     public static final long FILE_END = 2;
+    
+    public static final long FILE_ATTRIBUTE_NORMAL = 0x00000080L;
 
     /** Wrapper around CreateFile() on Windows */
     public static native FileDescriptor createFile(String path,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
index b78d791..bd25b9d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
@@ -649,6 +649,18 @@ abstract public class Shell {
     }
   }
   
+  public interface CommandExecutor {
+
+    void execute() throws IOException;
+
+    int getExitCode() throws IOException;
+
+    String getOutput() throws IOException;
+
+    void close();
+    
+  }
+  
   /**
    * A simple shell command executor.
    * 
@@ -657,7 +669,8 @@ abstract public class Shell {
    * directory and the environment remains unchanged. The output of the command 
    * is stored as-is and is expected to be small.
    */
-  public static class ShellCommandExecutor extends Shell {
+  public static class ShellCommandExecutor extends Shell 
+      implements CommandExecutor {
     
     private String[] command;
     private StringBuffer output;
@@ -749,6 +762,10 @@ abstract public class Shell {
       }
       return builder.toString();
     }
+
+    @Override
+    public void close() {
+    }
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj b/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
index 0d67e1e..2d60e56 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
+++ b/hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
@@ -107,6 +107,9 @@
       <AdditionalOptions Condition="'$(SnappyEnabled)' == 'true'">/D HADOOP_SNAPPY_LIBRARY=L\"snappy.dll\"</AdditionalOptions>
     </ClCompile>
     <ClCompile Include="src\org\apache\hadoop\util\NativeCrc32.c" />
+    <ClCompile Include="src\org\apache\hadoop\yarn\server\nodemanager\windows_secure_container_executor.c">
+      <AdditionalIncludeDirectories>src\org\apache\hadoop\io\nativeio;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+    </ClCompile>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\src\org\apache\hadoop\util\crc32c_tables.h" />
@@ -120,6 +123,7 @@
     <ClInclude Include="src\org\apache\hadoop\util\crc32c_tables.h" />
     <ClInclude Include="src\org\apache\hadoop\util\crc32_zlib_polynomial_tables.h" />
     <ClInclude Include="src\org_apache_hadoop.h" />
+    <ClInclude Include="src\org\apache\hadoop\yarn\server\nodemanager\windows_secure_container_executor.h" />
   </ItemGroup>
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />
   <ImportGroup Label="ExtensionTargets">

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
index f19d6be..792205d 100644
--- a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
@@ -71,8 +71,13 @@ static jmethodID nioe_ctor;
 // Please see HADOOP-7156 for details.
 jobject pw_lock_object;
 
+/*
+ * Throw a java.IO.IOException, generating the message from errno.
+ * NB. this is also used form windows_secure_container_executor.c
+ */
+extern void throw_ioe(JNIEnv* env, int errnum);
+
 // Internal functions
-static void throw_ioe(JNIEnv* env, int errnum);
 #ifdef UNIX
 static ssize_t get_pw_buflen();
 #endif
@@ -216,7 +221,7 @@ static int map_fadvise_flag(jint flag) {
  */
 JNIEXPORT void JNICALL
 Java_org_apache_hadoop_io_nativeio_NativeIO_initNative(
-	JNIEnv *env, jclass clazz) {
+  JNIEnv *env, jclass clazz) {
   stat_init(env, clazz);
   PASS_EXCEPTIONS_GOTO(env, error);
   nioe_init(env);
@@ -802,7 +807,7 @@ cleanup:
 /*
  * Throw a java.IO.IOException, generating the message from errno.
  */
-static void throw_ioe(JNIEnv* env, int errnum)
+void throw_ioe(JNIEnv* env, int errnum)
 {
 #ifdef UNIX
   char message[80];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/yarn/server/nodemanager/windows_secure_container_executor.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/yarn/server/nodemanager/windows_secure_container_executor.c b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/yarn/server/nodemanager/windows_secure_container_executor.c
new file mode 100644
index 0000000..7e65065
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/yarn/server/nodemanager/windows_secure_container_executor.c
@@ -0,0 +1,569 @@
+/**
+* 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 <jni.h>
+#include "org_apache_hadoop.h"
+#include "windows_secure_container_executor.h"
+#include "winutils.h"
+#include "file_descriptor.h"
+
+// class of org.apache.hadoop.yarn.server.nodemanager.WindowsSecureContainerExecutor.Native.WinutilsProcessStub
+static jclass wps_class = NULL;
+
+/*
+ * private static native void initWsceNative();
+ *
+ * We rely on this function rather than lazy initialization because
+ * the lazy approach may have a race if multiple callers try to
+ * init at the same time.
+ */
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_yarn_server_nodemanager_WindowsSecureContainerExecutor_00024Native_initWsceNative(
+  JNIEnv *env, jclass clazz) {
+
+#ifdef WINDOWS
+  winutils_process_stub_init(env);
+  PASS_EXCEPTIONS_GOTO(env, error);
+#endif
+
+  return;
+error:
+  // these are all idempodent and safe to call even if the
+  // class wasn't initted yet
+#ifdef WINDOWS
+  winutils_process_stub_deinit(env);
+#endif
+}
+
+static jmethodID wps_constructor = NULL;
+static jfieldID wps_hProcess = NULL;
+static jfieldID wps_hThread = NULL;
+static jfieldID wps_disposed = NULL;
+
+extern void throw_ioe(JNIEnv* env, int errnum);
+
+void winutils_process_stub_init(JNIEnv *env) {
+  if (wps_class != NULL) return; // already initted
+
+  wps_class = (*env)->FindClass(env, WINUTILS_PROCESS_STUB_CLASS);
+  PASS_EXCEPTIONS(env);
+
+  wps_class = (*env)->NewGlobalRef(env, wps_class);
+  PASS_EXCEPTIONS(env);
+
+  wps_hProcess = (*env)->GetFieldID(env, wps_class, "hProcess", "J");
+  PASS_EXCEPTIONS(env);  
+
+  wps_hThread = (*env)->GetFieldID(env, wps_class, "hThread", "J");
+  PASS_EXCEPTIONS(env);  
+
+  wps_disposed = (*env)->GetFieldID(env, wps_class, "disposed", "Z");
+  PASS_EXCEPTIONS(env); 
+
+  wps_constructor = (*env)->GetMethodID(env, wps_class, "<init>", "(JJJJJ)V");
+  PASS_EXCEPTIONS(env);
+
+  LogDebugMessage(L"winutils_process_stub_init\n");
+}
+
+void winutils_process_stub_deinit(JNIEnv *env) {
+  if (wps_class != NULL) {
+    (*env)->DeleteGlobalRef(env, wps_class);
+    wps_class = NULL;
+  }
+  wps_hProcess = NULL;
+  wps_hThread = NULL;
+  wps_disposed = NULL;
+  wps_constructor = NULL;
+  LogDebugMessage(L"winutils_process_stub_deinit\n");
+}
+
+jobject winutils_process_stub_create(JNIEnv *env, 
+  jlong hProcess, jlong hThread, jlong hStdIn, jlong hStdOut, jlong hStdErr) {
+  jobject obj = (*env)->NewObject(env, wps_class, wps_constructor, 
+    hProcess, hThread, hStdIn, hStdOut, hStdErr);
+  PASS_EXCEPTIONS_RET(env, NULL);
+
+  LogDebugMessage(L"winutils_process_stub_create: %p\n", obj);
+
+  return obj;
+}
+
+
+/*
+ * Class:     org.apache.hadoop.yarn.server.nodemanager.WindowsSecureContainerExecutor$Native
+ * Method:    createTaskAsUser
+ * Signature: (Ljava/lang/String;Ljava/lang/String;Ljava/lang/String;Ljava/lang/String)Lorg/apache/hadoop/io/nativeio/NativeIO$WinutilsProcessStub
+ */
+JNIEXPORT jobject JNICALL
+Java_org_apache_hadoop_yarn_server_nodemanager_WindowsSecureContainerExecutor_00024Native_createTaskAsUser0(JNIEnv* env,
+  jclass clazz, jstring jcwd, jstring jjobName, jstring juser, jstring jpidFile, jstring jcmdLine) {
+#ifdef UNIX
+    THROW(env, "java/io/IOException",
+      "The function createTaskAsUser is not supported on Unix");
+    return NULL;
+#endif
+
+#ifdef WINDOWS
+  LPCWSTR cwd = NULL, jobName = NULL, 
+    user = NULL, pidFile = NULL, cmdLine = NULL;
+  DWORD dwError = ERROR_SUCCESS;
+  HANDLE hProcess = INVALID_HANDLE_VALUE, 
+     hThread = INVALID_HANDLE_VALUE, 
+     hStdIn = INVALID_HANDLE_VALUE, 
+     hStdOut = INVALID_HANDLE_VALUE, 
+     hStdErr = INVALID_HANDLE_VALUE;
+  jobject ret = NULL;
+
+  cwd = (LPCWSTR) (*env)->GetStringChars(env, jcwd, NULL);
+  if (!cwd) goto done; // exception was thrown
+
+  jobName = (LPCWSTR) (*env)->GetStringChars(env, jjobName, NULL);
+  if (!jobName) goto done; // exception was thrown
+
+  user = (LPCWSTR) (*env)->GetStringChars(env, juser, NULL);
+  if (!user) goto done; // exception was thrown
+
+  pidFile = (LPCWSTR) (*env)->GetStringChars(env, jpidFile, NULL);
+  if (!pidFile) goto done; // exception was thrown
+
+  cmdLine = (LPCWSTR) (*env)->GetStringChars(env, jcmdLine, NULL);
+  if (!cmdLine) goto done; // exception was thrown
+
+  LogDebugMessage(L"createTaskAsUser: jcwd:%s job:%s juser:%s pid:%s cmd:%s\n",
+    cwd, jobName, user, pidFile, cmdLine);
+  
+  dwError = RpcCall_TaskCreateAsUser(cwd, jobName, user, pidFile, cmdLine, 
+    &hProcess, &hThread, &hStdIn, &hStdOut, &hStdErr);
+
+  if (ERROR_SUCCESS == dwError) {
+    ret = winutils_process_stub_create(env, (jlong) hProcess, (jlong) hThread,
+      (jlong) hStdIn, (jlong) hStdOut, (jlong) hStdErr);
+
+    if (NULL == ret) {
+      TerminateProcess(hProcess, EXIT_FAILURE);
+      CloseHandle(hThread);
+      CloseHandle(hProcess);
+      CloseHandle(hStdIn);
+      CloseHandle(hStdOut);
+      CloseHandle(hStdErr);
+    }
+  }
+
+
+  if (dwError != ERROR_SUCCESS) {
+    throw_ioe (env, dwError);
+  }
+
+done:
+
+  if (cwd)     (*env)->ReleaseStringChars(env, jcwd, cwd);
+  if (jobName) (*env)->ReleaseStringChars(env, jjobName, jobName);
+  if (user)    (*env)->ReleaseStringChars(env, juser, user);
+  if (pidFile) (*env)->ReleaseStringChars(env, jpidFile, pidFile);
+  if (cmdLine) (*env)->ReleaseStringChars(env, jcmdLine, cmdLine);
+
+  return ret;
+  
+#endif
+}
+
+/*
+ * Class:     org.apache.hadoop.yarn.server.nodemanager.WindowsSecureContainerExecutor$Native$Elevated
+ * Method:    elevatedKillTaskImpl
+ * Signature: (Ljava/lang/String;Ljava/lang/String;Ljava/lang/String;)V
+ */
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_yarn_server_nodemanager_WindowsSecureContainerExecutor_00024Native_00024Elevated_elevatedKillTaskImpl(JNIEnv* env,
+  jclass clazz, jstring jtask_name) {
+#ifdef UNIX
+    THROW(env, "java/io/IOException",
+      "The function elevatedSetOwner0 is not supported on Unix");
+    return;
+#endif
+
+#ifdef WINDOWS
+
+  LPCWSTR task_name = NULL;
+  DWORD dwError;
+
+  task_name = (LPCWSTR) (*env)->GetStringChars(env, jtask_name, NULL);
+  if (!task_name) goto done; // exception was thrown
+
+
+  dwError = RpcCall_WinutilsKillTask(task_name);
+
+  if (dwError != ERROR_SUCCESS) {
+    throw_ioe (env, dwError);
+  }
+
+done:
+  if (task_name)     (*env)->ReleaseStringChars(env, jtask_name, task_name);
+
+#endif
+
+}
+
+
+/*
+ * Class:     org.apache.hadoop.yarn.server.nodemanager.WindowsSecureContainerExecutor$Native$Elevated
+ * Method:    elevatedChownImpl
+ * Signature: (Ljava/lang/String;Ljava/lang/String;Ljava/lang/String;)V
+ */
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_yarn_server_nodemanager_WindowsSecureContainerExecutor_00024Native_00024Elevated_elevatedChownImpl(JNIEnv* env,
+  jclass clazz, jstring jpath, jstring juser, jstring jgroup) {
+#ifdef UNIX
+    THROW(env, "java/io/IOException",
+      "The function elevatedSetOwner0 is not supported on Unix");
+    return;
+#endif
+
+#ifdef WINDOWS
+
+  LPCWSTR path = NULL, user = NULL, group = NULL;
+  DWORD dwError;
+
+  path = (LPCWSTR) (*env)->GetStringChars(env, jpath, NULL);
+  if (!path) goto done; // exception was thrown
+
+  if (juser) {
+    user = (LPCWSTR) (*env)->GetStringChars(env, juser, NULL);
+    if (!user) goto done; // exception was thrown
+  }
+
+  if (jgroup) {
+    group = (LPCWSTR) (*env)->GetStringChars(env, jgroup, NULL);
+    if (!group) goto done; // exception was thrown
+  }
+
+  dwError = RpcCall_WinutilsChown(path, user, group);
+
+  if (dwError != ERROR_SUCCESS) {
+    throw_ioe (env, dwError);
+  }
+
+done:
+  if (path)     (*env)->ReleaseStringChars(env, jpath, path);
+  if (user)     (*env)->ReleaseStringChars(env, juser, user);
+  if (group)     (*env)->ReleaseStringChars(env, jgroup, group);
+
+#endif
+
+}
+
+
+/*
+ * Class:     org.apache.hadoop.yarn.server.nodemanager.WindowsSecureContainerExecutor$Native$Elevated
+ * Method:    elevatedMkDirImpl
+ * Signature: (Ljava/lang/String;)V
+ */
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_yarn_server_nodemanager_WindowsSecureContainerExecutor_00024Native_00024Elevated_elevatedMkDirImpl(JNIEnv* env,
+  jclass clazz, jstring jpath) {
+#ifdef UNIX
+    THROW(env, "java/io/IOException",
+      "The function elevatedMkDirImpl is not supported on Unix");
+    return;
+#endif
+
+#ifdef WINDOWS
+
+  LPCWSTR path = NULL, user = NULL, group = NULL;
+  DWORD dwError;
+
+  path = (LPCWSTR) (*env)->GetStringChars(env, jpath, NULL);
+  if (!path) goto done; // exception was thrown
+
+  dwError = RpcCall_WinutilsMkDir(path);
+
+  if (dwError != ERROR_SUCCESS) {
+    throw_ioe (env, dwError);
+  }
+
+done:
+  if (path)     (*env)->ReleaseStringChars(env, jpath, path);
+
+#endif
+
+}
+
+
+/*
+ * Class:     org.apache.hadoop.yarn.server.nodemanager.WindowsSecureContainerExecutor$Native$Elevated
+ * Method:    elevatedChmodImpl
+ * Signature: (Ljava/lang/String;I)V
+ */
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_yarn_server_nodemanager_WindowsSecureContainerExecutor_00024Native_00024Elevated_elevatedChmodImpl(JNIEnv* env,
+  jclass clazz, jstring jpath, jint jmode) {
+#ifdef UNIX
+    THROW(env, "java/io/IOException",
+      "The function elevatedChmodImpl is not supported on Unix");
+    return;
+#endif
+
+#ifdef WINDOWS
+
+  LPCWSTR path = NULL;
+  DWORD dwError;
+
+  path = (LPCWSTR) (*env)->GetStringChars(env, jpath, NULL);
+  if (!path) goto done; // exception was thrown
+
+  dwError = RpcCall_WinutilsChmod(path, (int) jmode);
+
+  if (dwError != ERROR_SUCCESS) {
+    throw_ioe (env, dwError);
+  }
+
+done:
+  if (path)     (*env)->ReleaseStringChars(env, jpath, path);
+
+#endif
+
+}
+
+
+/*
+ * Class:     org.apache.hadoop.yarn.server.nodemanager.WindowsSecureContainerExecutor$Native$Elevated
+ * Method:    elevatedCopyImpl
+ * Signature: (I;Ljava/lang/String;Ljava/lang/String;Ljava/lang/String;Z)V
+ */
+JNIEXPORT void JNICALL
+Java_org_apache_hadoop_yarn_server_nodemanager_WindowsSecureContainerExecutor_00024Native_00024Elevated_elevatedCopyImpl(JNIEnv* env,
+  jclass clazz, jint joperation, jstring jsourcePath, jstring jdestinationPath, jboolean replaceExisting) {
+#ifdef UNIX
+    THROW(env, "java/io/IOException",
+      "The function elevatedCopyImpl is not supported on Unix");
+    return;
+#endif
+
+#ifdef WINDOWS
+
+  LPCWSTR sourcePath = NULL, destinationPath = NULL;
+  DWORD dwError;
+
+  sourcePath = (LPCWSTR) (*env)->GetStringChars(env, jsourcePath, NULL);
+  if (!sourcePath) goto done; // exception was thrown
+
+  destinationPath = (LPCWSTR) (*env)->GetStringChars(env, jdestinationPath, NULL);
+  if (!destinationPath) goto done; // exception was thrown
+
+  dwError = RpcCall_WinutilsMoveFile((INT) joperation, sourcePath, destinationPath, (BOOL) replaceExisting);
+
+  if (dwError != ERROR_SUCCESS) {
+    throw_ioe (env, dwError);
+  }
+
+done:
+  if (sourcePath)     (*env)->ReleaseStringChars(env, jsourcePath, sourcePath);
+  if (destinationPath)     (*env)->ReleaseStringChars(env, jdestinationPath, destinationPath);
+#endif
+}
+
+/*
+ * Class:     org.apache.hadoop.yarn.server.nodemanager.WindowsSecureContainerExecutor$Native$Elevated
+ * Method:    elevatedCreateImpl
+ * Signature: (Ljava/lang/String;J;J;J;J)J
+ */
+JNIEXPORT jlong JNICALL
+Java_org_apache_hadoop_yarn_server_nodemanager_WindowsSecureContainerExecutor_00024Native_00024Elevated_elevatedCreateImpl(JNIEnv* env,
+  jclass clazz, jstring jpath, jlong jdesired_access, jlong jshare_mode, jlong jcreation_disposition, jlong jflags) {
+#ifdef UNIX
+    THROW(env, "java/io/IOException",
+      "The function elevatedCreateImpl is not supported on Unix");
+    return 0;
+#endif
+
+#ifdef WINDOWS
+
+  LPCWSTR path = NULL;
+  DWORD dwError;
+  HANDLE hFile = INVALID_HANDLE_VALUE;
+
+  path = (LPCWSTR) (*env)->GetStringChars(env, jpath, NULL);
+  if (!path) goto done; // exception was thrown
+
+  dwError = RpcCall_WinutilsCreateFile(path, 
+    (DWORD) jdesired_access, (DWORD) jshare_mode, (DWORD) jcreation_disposition, (DWORD) jflags,
+    &hFile);
+
+  if (dwError != ERROR_SUCCESS) {
+    throw_ioe (env, dwError);
+  }
+
+done:
+  if (path)     (*env)->ReleaseStringChars(env, jpath, path);
+  return hFile;
+#endif
+}
+
+/*
+ * Class:     org.apache.hadoop.yarn.server.nodemanager.WindowsSecureContainerExecutor$Native$Elevated
+ * Method:    elevatedDeletePathImpl
+ * Signature: (Ljava/lang/String;Z)Z
+ */
+JNIEXPORT jboolean JNICALL
+Java_org_apache_hadoop_yarn_server_nodemanager_WindowsSecureContainerExecutor_00024Native_00024Elevated_elevatedDeletePathImpl(JNIEnv* env,
+  jclass clazz, jstring jpath, jboolean jIsDir) {
+#ifdef UNIX
+    THROW(env, "java/io/IOException",
+      "The function elevatedDeleteFileImpl is not supported on Unix");
+    return  JNI_FALSE;
+#endif
+
+#ifdef WINDOWS
+
+  LPCWSTR path = NULL;
+  DWORD dwError;
+  BOOL deleted = FALSE;
+
+  path = (LPCWSTR) (*env)->GetStringChars(env, jpath, NULL);
+  if (!path) goto done; // exception was thrown
+
+  dwError = RpcCall_WinutilsDeletePath(path, (BOOL) jIsDir, &deleted);
+
+  if (dwError != ERROR_SUCCESS) {
+    throw_ioe (env, dwError);
+  }
+
+done:
+  if (path)     (*env)->ReleaseStringChars(env, jpath, path);
+  return (jboolean) deleted;
+#endif
+}
+
+
+
+
+/*
+ * native void destroy();
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT void JNICALL 
+Java_org_apache_hadoop_yarn_server_nodemanager_WindowsSecureContainerExecutor_00024Native_00024WinutilsProcessStub_destroy(
+  JNIEnv *env, jobject objSelf) {
+
+  HANDLE hProcess = (HANDLE)(*env)->GetLongField(env, objSelf, wps_hProcess);
+  LogDebugMessage(L"TerminateProcess: %x\n", hProcess);
+  TerminateProcess(hProcess, EXIT_FAILURE);
+}
+
+/*
+ * native void waitFor();
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT void JNICALL 
+Java_org_apache_hadoop_yarn_server_nodemanager_WindowsSecureContainerExecutor_00024Native_00024WinutilsProcessStub_waitFor(
+  JNIEnv *env, jobject objSelf) {
+
+  HANDLE hProcess = (HANDLE)(*env)->GetLongField(env, objSelf, wps_hProcess);
+  LogDebugMessage(L"WaitForSingleObject: %x\n", hProcess);
+  WaitForSingleObject(hProcess, INFINITE);
+}
+
+
+
+/*
+ * native void resume();
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT void JNICALL 
+Java_org_apache_hadoop_yarn_server_nodemanager_WindowsSecureContainerExecutor_00024Native_00024WinutilsProcessStub_resume(
+  JNIEnv *env, jobject objSelf) {
+
+  DWORD dwError;
+  HANDLE hThread = (HANDLE)(*env)->GetLongField(env, objSelf, wps_hThread);
+  if (-1 == ResumeThread(hThread)) {
+    dwError = GetLastError();
+    LogDebugMessage(L"ResumeThread: %x error:%d\n", hThread, dwError);
+    throw_ioe(env, dwError);
+  }
+}
+
+/*
+ * native int exitValue();
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT jint JNICALL 
+Java_org_apache_hadoop_yarn_server_nodemanager_WindowsSecureContainerExecutor_00024Native_00024WinutilsProcessStub_exitValue(
+  JNIEnv *env, jobject objSelf) {
+
+  DWORD exitCode;
+  DWORD dwError;
+  HANDLE hProcess = (HANDLE)(*env)->GetLongField(env, objSelf, wps_hProcess);
+  if (!GetExitCodeProcess(hProcess, &exitCode)) {
+    dwError = GetLastError();
+    throw_ioe(env, dwError);
+    return dwError; // exception was thrown, return value doesn't really matter
+  }
+  LogDebugMessage(L"GetExitCodeProcess: %x :%d\n", hProcess, exitCode);
+  
+  return exitCode;
+}
+
+
+/*
+ * native void dispose();
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT void JNICALL 
+Java_org_apache_hadoop_yarn_server_nodemanager_WindowsSecureContainerExecutor_00024Native_00024WinutilsProcessStub_dispose(
+  JNIEnv *env, jobject objSelf) {
+
+  HANDLE hProcess = INVALID_HANDLE_VALUE, 
+         hThread  = INVALID_HANDLE_VALUE;
+
+  jboolean disposed = (*env)->GetBooleanField(env, objSelf, wps_disposed);
+
+  if (JNI_TRUE != disposed) {
+    hProcess = (HANDLE)(*env)->GetLongField(env, objSelf, wps_hProcess);
+    hThread = (HANDLE)(*env)->GetLongField(env, objSelf, wps_hThread);
+
+    CloseHandle(hProcess);
+    CloseHandle(hThread);
+    (*env)->SetBooleanField(env, objSelf, wps_disposed, JNI_TRUE);
+    LogDebugMessage(L"disposed: %p\n", objSelf);
+  }
+}
+
+
+/*
+ * native static FileDescriptor getFileDescriptorFromHandle(long handle);
+ *
+ * The "00024" in the function name is an artifact of how JNI encodes
+ * special characters. U+0024 is '$'.
+ */
+JNIEXPORT jobject JNICALL 
+Java_org_apache_hadoop_yarn_server_nodemanager_WindowsSecureContainerExecutor_00024Native_00024WinutilsProcessStub_getFileDescriptorFromHandle(
+  JNIEnv *env, jclass klass, jlong handle) {
+
+  LogDebugMessage(L"getFileDescriptorFromHandle: %x\n", handle);
+  return fd_create(env, (long) handle);
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/yarn/server/nodemanager/windows_secure_container_executor.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/yarn/server/nodemanager/windows_secure_container_executor.h b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/yarn/server/nodemanager/windows_secure_container_executor.h
new file mode 100644
index 0000000..381b424
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/yarn/server/nodemanager/windows_secure_container_executor.h
@@ -0,0 +1,27 @@
+/*
+ *  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.
+ */
+#pragma once
+
+
+#define WINUTILS_PROCESS_STUB_CLASS "org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor$Native$WinutilsProcessStub"
+
+void winutils_process_stub_init(JNIEnv *env);
+void winutils_process_stub_deinit(JNIEnv *env);
+jobject winutils_process_stub_create(JNIEnv *env, 
+  jlong hProcess, jlong hThread, jlong hStdIn, jlong hStdOut, jlong hStdErr);
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/src/main/winutils/chown.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/chown.c b/hadoop-common-project/hadoop-common/src/main/winutils/chown.c
index 1be8121..d124f73 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/chown.c
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/chown.c
@@ -18,93 +18,6 @@
 #include "winutils.h"
 
 //----------------------------------------------------------------------------
-// Function: ChangeFileOwnerBySid
-//
-// Description:
-//  Change a file or directory ownership by giving new owner and group SIDs
-//
-// Returns:
-//  ERROR_SUCCESS: on success
-//  Error code: otherwise
-//
-// Notes:
-//  This function is long path safe, i.e. the path will be converted to long
-//  path format if not already converted. So the caller does not need to do
-//  the converstion before calling the method.
-//
-static DWORD ChangeFileOwnerBySid(__in LPCWSTR path,
-  __in_opt PSID pNewOwnerSid, __in_opt PSID pNewGroupSid)
-{
-  LPWSTR longPathName = NULL;
-  INT oldMode = 0;
-
-  SECURITY_INFORMATION securityInformation = 0;
-
-  DWORD dwRtnCode = ERROR_SUCCESS;
-
-  // Convert the path the the long path
-  //
-  dwRtnCode = ConvertToLongPath(path, &longPathName);
-  if (dwRtnCode != ERROR_SUCCESS)
-  {
-    goto ChangeFileOwnerByNameEnd;
-  }
-
-  // Get a pointer to the existing owner information and DACL
-  //
-  dwRtnCode = FindFileOwnerAndPermission(longPathName, FALSE, NULL, NULL, &oldMode);
-  if (dwRtnCode != ERROR_SUCCESS)
-  {
-    goto ChangeFileOwnerByNameEnd;
-  }
-
-  // We need SeTakeOwnershipPrivilege to set the owner if the caller does not
-  // have WRITE_OWNER access to the object; we need SeRestorePrivilege if the
-  // SID is not contained in the caller's token, and have the SE_GROUP_OWNER
-  // permission enabled.
-  //
-  if (EnablePrivilege(L"SeTakeOwnershipPrivilege") != ERROR_SUCCESS)
-  {
-    fwprintf(stdout, L"INFO: The user does not have SeTakeOwnershipPrivilege.\n");
-  }
-  if (EnablePrivilege(L"SeRestorePrivilege") != ERROR_SUCCESS)
-  {
-    fwprintf(stdout, L"INFO: The user does not have SeRestorePrivilege.\n");
-  }
-
-  assert(pNewOwnerSid != NULL || pNewGroupSid != NULL);
-
-  // Set the owners of the file.
-  //
-  if (pNewOwnerSid != NULL) securityInformation |= OWNER_SECURITY_INFORMATION;
-  if (pNewGroupSid != NULL) securityInformation |= GROUP_SECURITY_INFORMATION;
-  dwRtnCode = SetNamedSecurityInfoW(
-    longPathName,
-    SE_FILE_OBJECT,
-    securityInformation,
-    pNewOwnerSid,
-    pNewGroupSid,
-    NULL,
-    NULL);
-  if (dwRtnCode != ERROR_SUCCESS)
-  {
-    goto ChangeFileOwnerByNameEnd;
-  }
-
-  // Set the permission on the file for the new owner.
-  //
-  dwRtnCode = ChangeFileModeByMask(longPathName, oldMode);
-  if (dwRtnCode != ERROR_SUCCESS)
-  {
-    goto ChangeFileOwnerByNameEnd;
-  }
-
-ChangeFileOwnerByNameEnd:
-  LocalFree(longPathName);
-  return dwRtnCode;
-}
-
-//----------------------------------------------------------------------------
 // Function: Chown
 //
 // Description:
@@ -130,9 +43,6 @@ int Chown(__in int argc, __in_ecount(argc) wchar_t *argv[])
   LPWSTR groupName = NULL;
   size_t groupNameLen = 0;
 
-  PSID pNewOwnerSid = NULL;
-  PSID pNewGroupSid = NULL;
-
   DWORD dwRtnCode = 0;
 
   int ret = EXIT_FAILURE;
@@ -210,48 +120,16 @@ int Chown(__in int argc, __in_ecount(argc) wchar_t *argv[])
     goto ChownEnd;
   }
 
-  if (userName != NULL)
-  {
-    dwRtnCode = GetSidFromAcctNameW(userName, &pNewOwnerSid);
-    if (dwRtnCode != ERROR_SUCCESS)
-    {
-      ReportErrorCode(L"GetSidFromAcctName", dwRtnCode);
-      fwprintf(stderr, L"Invalid user name: %s\n", userName);
-      goto ChownEnd;
-    }
-  }
-
-  if (groupName != NULL)
-  {
-    dwRtnCode = GetSidFromAcctNameW(groupName, &pNewGroupSid);
-    if (dwRtnCode != ERROR_SUCCESS)
-    {
-      ReportErrorCode(L"GetSidFromAcctName", dwRtnCode);
-      fwprintf(stderr, L"Invalid group name: %s\n", groupName);
-      goto ChownEnd;
-    }
-  }
-
-  if (wcslen(pathName) == 0 || wcsspn(pathName, L"/?|><:*\"") != 0)
-  {
-    fwprintf(stderr, L"Incorrect file name format: %s\n", pathName);
-    goto ChownEnd;
-  }
-
-  dwRtnCode = ChangeFileOwnerBySid(pathName, pNewOwnerSid, pNewGroupSid);
-  if (dwRtnCode != ERROR_SUCCESS)
-  {
-    ReportErrorCode(L"ChangeFileOwnerBySid", dwRtnCode);
-    goto ChownEnd;
-  }
+	dwRtnCode = ChownImpl(userName, groupName, pathName);
+	if (dwRtnCode) {
+		goto ChownEnd;
+	}
 
   ret = EXIT_SUCCESS;
 
 ChownEnd:
   LocalFree(userName);
   LocalFree(groupName);
-  LocalFree(pNewOwnerSid);
-  LocalFree(pNewGroupSid);
 
   return ret;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/src/main/winutils/client.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/client.c b/hadoop-common-project/hadoop-common/src/main/winutils/client.c
new file mode 100644
index 0000000..047bfb5
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/client.c
@@ -0,0 +1,498 @@
+/**
+* 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 "winutils.h"
+#include <Winsvc.h>
+#include <errno.h>
+#include "hadoopwinutilsvc_h.h"
+
+#pragma comment(lib, "Rpcrt4.lib")
+#pragma comment(lib, "advapi32.lib")
+
+static ACCESS_MASK CLIENT_MASK = 1;
+
+VOID ReportClientError(LPWSTR lpszLocation, DWORD dwError) {
+  LPWSTR      debugMsg = NULL;
+  int         len;
+  WCHAR       hexError[32];
+  HRESULT     hr;
+
+  if (IsDebuggerPresent()) {
+    len = FormatMessageW(
+      FORMAT_MESSAGE_ALLOCATE_BUFFER | FORMAT_MESSAGE_FROM_SYSTEM,
+      NULL, dwError,
+      MAKELANGID(LANG_NEUTRAL, SUBLANG_DEFAULT),
+      (LPWSTR)&debugMsg, 0, NULL);
+
+    LogDebugMessage(L"%s: %s: %x: %.*s\n", GetSystemTimeString(), lpszLocation, dwError, len, debugMsg);
+  }
+
+  if (NULL != debugMsg)  LocalFree(debugMsg);
+}
+
+DWORD PrepareRpcBindingHandle(
+  __out RPC_BINDING_HANDLE* pHadoopWinutilsSvcBinding) {
+  DWORD       dwError = EXIT_FAILURE;
+  RPC_STATUS  status;
+  LPWSTR      lpszStringBinding    = NULL;
+  ULONG       ulCode;
+  RPC_SECURITY_QOS_V3 qos;
+  SID_IDENTIFIER_AUTHORITY authNT = SECURITY_NT_AUTHORITY;
+  BOOL rpcBindingInit = FALSE;
+  PSID        pLocalSystemSid = NULL;
+  DWORD       cbSystemSidSize = SECURITY_MAX_SID_SIZE;
+
+  pLocalSystemSid = (PSID) LocalAlloc(LPTR, cbSystemSidSize);
+  if (!pLocalSystemSid) {
+    dwError = GetLastError();
+    ReportClientError(L"LocalAlloc", dwError);
+    goto done;
+  }
+
+  if (!CreateWellKnownSid(WinLocalSystemSid, NULL, pLocalSystemSid, &cbSystemSidSize)) {
+    dwError = GetLastError();
+    ReportClientError(L"CreateWellKnownSid", dwError);
+    goto done;
+  }
+
+  ZeroMemory(&qos, sizeof(qos));
+  qos.Version = RPC_C_SECURITY_QOS_VERSION_3;
+  qos.Capabilities = RPC_C_QOS_CAPABILITIES_LOCAL_MA_HINT |  RPC_C_QOS_CAPABILITIES_MUTUAL_AUTH;
+  qos.IdentityTracking = RPC_C_QOS_IDENTITY_DYNAMIC;
+  qos.ImpersonationType = RPC_C_IMP_LEVEL_DEFAULT;
+  qos.Sid = pLocalSystemSid;
+
+  status = RpcStringBindingCompose(NULL,
+                 SVCBINDING,
+                 NULL,
+                 SVCNAME,
+                 NULL,
+                 &lpszStringBinding);
+  if (RPC_S_OK != status) {
+    ReportClientError(L"RpcStringBindingCompose", status);
+    dwError = status;
+    goto done;
+  }
+
+  status = RpcBindingFromStringBinding(lpszStringBinding, pHadoopWinutilsSvcBinding);
+
+  if (RPC_S_OK != status) {
+    ReportClientError(L"RpcBindingFromStringBinding", status);
+    dwError = status;
+    goto done;
+  }
+  rpcBindingInit = TRUE;
+
+  status = RpcBindingSetAuthInfoEx(
+                  *pHadoopWinutilsSvcBinding,
+                  NULL,
+                  RPC_C_AUTHN_LEVEL_PKT_PRIVACY,  // AuthnLevel
+                  RPC_C_AUTHN_WINNT,              // AuthnSvc
+                  NULL,                           // AuthnIdentity (self)
+                  RPC_C_AUTHZ_NONE,               // AuthzSvc
+                  &qos);
+  if (RPC_S_OK != status) {
+    ReportClientError(L"RpcBindingSetAuthInfoEx", status);
+    dwError = status;
+    goto done;
+  }
+
+  dwError = ERROR_SUCCESS;
+
+done:
+
+  if (dwError && rpcBindingInit) RpcBindingFree(pHadoopWinutilsSvcBinding);
+
+  if (pLocalSystemSid) LocalFree(pLocalSystemSid);
+  
+  if (NULL != lpszStringBinding) {
+    status = RpcStringFree(&lpszStringBinding);
+    if (RPC_S_OK != status) {
+      ReportClientError(L"RpcStringFree", status);
+    }
+  }
+  
+  return dwError;
+}
+
+DWORD RpcCall_WinutilsKillTask(
+  __in LPCWSTR taskName) {
+
+  DWORD       dwError = EXIT_FAILURE;
+  ULONG       ulCode;
+  KILLTASK_REQUEST request;
+  RPC_BINDING_HANDLE hHadoopWinutilsSvcBinding;
+  BOOL rpcBindingInit = FALSE;
+
+  dwError = PrepareRpcBindingHandle(&hHadoopWinutilsSvcBinding);
+  if (dwError) {
+    ReportClientError(L"PrepareRpcBindingHandle", dwError);
+    goto done;
+  }
+  rpcBindingInit = TRUE;
+
+  ZeroMemory(&request, sizeof(request));
+  request.taskName = taskName;
+
+  RpcTryExcept {
+    dwError = WinutilsKillTask(hHadoopWinutilsSvcBinding, &request);
+  }
+  RpcExcept(1) {
+    ulCode = RpcExceptionCode();
+    ReportClientError(L"RpcExcept", ulCode);
+    dwError = (DWORD) ulCode;
+  }
+  RpcEndExcept;
+
+done:
+  if (rpcBindingInit) RpcBindingFree(&hHadoopWinutilsSvcBinding);
+
+  LogDebugMessage(L"RpcCall_WinutilsKillTask: %s :%d\n", taskName, dwError);
+
+  return dwError;
+}
+
+DWORD RpcCall_WinutilsMkDir(
+  __in LPCWSTR filePath) {
+
+  DWORD       dwError = EXIT_FAILURE;
+  ULONG       ulCode;
+  MKDIR_REQUEST request;
+  RPC_BINDING_HANDLE hHadoopWinutilsSvcBinding;
+  BOOL rpcBindingInit = FALSE;
+
+  dwError = PrepareRpcBindingHandle(&hHadoopWinutilsSvcBinding);
+  if (dwError) {
+    ReportClientError(L"PrepareRpcBindingHandle", dwError);
+    goto done;
+  }
+  rpcBindingInit = TRUE;
+
+  ZeroMemory(&request, sizeof(request));
+  request.filePath = filePath;
+
+  RpcTryExcept {
+    dwError = WinutilsMkDir(hHadoopWinutilsSvcBinding, &request);
+  }
+  RpcExcept(1) {
+    ulCode = RpcExceptionCode();
+    ReportClientError(L"RpcExcept", ulCode);
+    dwError = (DWORD) ulCode;
+  }
+  RpcEndExcept;
+
+done:
+  if (rpcBindingInit) RpcBindingFree(&hHadoopWinutilsSvcBinding);
+
+  LogDebugMessage(L"RpcCall_WinutilsMkDir: %s :%d\n", filePath, dwError);
+
+  return dwError;
+}
+
+
+
+DWORD RpcCall_WinutilsChown(
+  __in LPCWSTR filePath, 
+  __in_opt LPCWSTR ownerName, 
+  __in_opt LPCWSTR groupName) {
+
+  DWORD       dwError = EXIT_FAILURE;
+  ULONG       ulCode;
+  CHOWN_REQUEST request;
+  RPC_BINDING_HANDLE hHadoopWinutilsSvcBinding;
+  BOOL rpcBindingInit = FALSE;
+
+  dwError = PrepareRpcBindingHandle(&hHadoopWinutilsSvcBinding);
+  if (dwError) {
+    ReportClientError(L"PrepareRpcBindingHandle", dwError);
+    goto done;
+  }
+  rpcBindingInit = TRUE;
+
+  ZeroMemory(&request, sizeof(request));
+  request.filePath = filePath;
+  request.ownerName = ownerName;
+  request.groupName = groupName;
+
+  RpcTryExcept {
+    dwError = WinutilsChown(hHadoopWinutilsSvcBinding, &request);
+  }
+  RpcExcept(1) {
+    ulCode = RpcExceptionCode();
+    ReportClientError(L"RpcExcept", ulCode);
+    dwError = (DWORD) ulCode;
+  }
+  RpcEndExcept;
+
+done:
+  if (rpcBindingInit) RpcBindingFree(&hHadoopWinutilsSvcBinding);
+
+  LogDebugMessage(L"RpcCall_WinutilsChown: %s %s %s :%d\n",
+    ownerName, groupName, filePath, dwError);
+
+  return dwError;
+}
+
+
+DWORD RpcCall_WinutilsChmod(
+  __in LPCWSTR filePath, 
+  __in int mode) {
+
+  DWORD       dwError = EXIT_FAILURE;
+  ULONG       ulCode;
+  CHMOD_REQUEST request;
+  RPC_BINDING_HANDLE hHadoopWinutilsSvcBinding;
+  BOOL rpcBindingInit = FALSE;
+
+  dwError = PrepareRpcBindingHandle(&hHadoopWinutilsSvcBinding);
+  if (dwError) {
+    ReportClientError(L"PrepareRpcBindingHandle", dwError);
+    goto done;
+  }
+  rpcBindingInit = TRUE;
+
+  ZeroMemory(&request, sizeof(request));
+  request.filePath = filePath;
+  request.mode = mode;
+
+  RpcTryExcept {
+    dwError = WinutilsChmod(hHadoopWinutilsSvcBinding, &request);
+  }
+  RpcExcept(1) {
+    ulCode = RpcExceptionCode();
+    ReportClientError(L"RpcExcept", ulCode);
+    dwError = (DWORD) ulCode;
+  }
+  RpcEndExcept;
+
+done:
+  if (rpcBindingInit) RpcBindingFree(&hHadoopWinutilsSvcBinding);
+
+  LogDebugMessage(L"RpcCall_WinutilsChmod: %s %o :%d\n",
+    filePath, mode, dwError);
+
+  return dwError;
+} 
+
+
+
+DWORD RpcCall_WinutilsMoveFile(
+  __in int operation,
+  __in LPCWSTR sourcePath, 
+  __in LPCWSTR destinationPath,
+  __in BOOL replaceExisting) {
+
+  DWORD       dwError = EXIT_FAILURE;
+  ULONG       ulCode;
+  MOVEFILE_REQUEST request;
+  RPC_BINDING_HANDLE hHadoopWinutilsSvcBinding;
+  BOOL rpcBindingInit = FALSE;
+
+  dwError = PrepareRpcBindingHandle(&hHadoopWinutilsSvcBinding);
+  if (dwError) {
+    ReportClientError(L"PrepareRpcBindingHandle", dwError);
+    goto done;
+  }
+  rpcBindingInit = TRUE;
+
+  ZeroMemory(&request, sizeof(request));
+  request.operation = operation;
+  request.sourcePath = sourcePath;
+  request.destinationPath = destinationPath;
+  request.replaceExisting = replaceExisting;
+
+  RpcTryExcept {
+    dwError = WinutilsMoveFile(hHadoopWinutilsSvcBinding, &request);
+  }
+  RpcExcept(1) {
+    ulCode = RpcExceptionCode();
+    ReportClientError(L"RpcExcept", ulCode);
+    dwError = (DWORD) ulCode;
+  }
+  RpcEndExcept;
+
+done:
+  if (rpcBindingInit) RpcBindingFree(&hHadoopWinutilsSvcBinding);
+
+  LogDebugMessage(L"RpcCall_WinutilsMoveFile: %s %s %d :%d\n",
+    sourcePath, destinationPath, replaceExisting, dwError);
+
+  return dwError;
+}
+
+DWORD RpcCall_WinutilsCreateFile(
+  __in LPCWSTR path,
+  __in DWORD desiredAccess,
+  __in DWORD shareMode,
+  __in DWORD creationDisposition,
+  __in DWORD flags,
+  __out HANDLE* hFile) {
+
+  DWORD       dwError = EXIT_FAILURE;
+  ULONG       ulCode;
+  DWORD       dwSelfPid = GetCurrentProcessId();
+  CREATEFILE_REQUEST request;
+  CREATEFILE_RESPONSE *response = NULL;
+  RPC_BINDING_HANDLE hHadoopWinutilsSvcBinding;
+  BOOL rpcBindingInit = FALSE;
+
+  dwError = PrepareRpcBindingHandle(&hHadoopWinutilsSvcBinding);
+  if (dwError) {
+    ReportClientError(L"PrepareRpcBindingHandle", dwError);
+    goto done;
+  }
+  rpcBindingInit = TRUE;
+
+  ZeroMemory(&request, sizeof(request));
+  request.path = path;
+  request.desiredAccess = desiredAccess;
+  request.shareMode = shareMode;
+  request.creationDisposition = creationDisposition;
+  request.flags = flags;
+
+  RpcTryExcept {
+    dwError = WinutilsCreateFile(hHadoopWinutilsSvcBinding, dwSelfPid, &request, &response);
+  }
+  RpcExcept(1) {
+    ulCode = RpcExceptionCode();
+    ReportClientError(L"RpcExcept", ulCode);
+    dwError = (DWORD) ulCode;
+  }
+  RpcEndExcept;
+
+  if (ERROR_SUCCESS == dwError) {
+    *hFile = response->hFile;
+  }
+
+done:
+  if (rpcBindingInit) RpcBindingFree(&hHadoopWinutilsSvcBinding);
+
+  if(NULL != response) MIDL_user_free(response);
+
+  LogDebugMessage(L"RpcCall_WinutilsCreateFile: %s %d, %d, %d, %d :%d\n",
+    path, desiredAccess, shareMode, creationDisposition, flags, dwError);
+
+  return dwError;
+}
+
+
+DWORD RpcCall_WinutilsDeletePath(
+  __in LPCWSTR    path,
+  __in BOOL       isDir,
+  __out BOOL*     pDeleted) {
+
+  DWORD       dwError = EXIT_FAILURE;
+  ULONG       ulCode;
+  DELETEPATH_REQUEST request;
+  DELETEPATH_RESPONSE *response = NULL;
+  RPC_BINDING_HANDLE hHadoopWinutilsSvcBinding;
+  BOOL rpcBindingInit = FALSE;
+
+  pDeleted = FALSE;
+
+  dwError = PrepareRpcBindingHandle(&hHadoopWinutilsSvcBinding);
+  if (dwError) {
+    ReportClientError(L"PrepareRpcBindingHandle", dwError);
+    goto done;
+  }
+  rpcBindingInit = TRUE;
+
+  ZeroMemory(&request, sizeof(request));
+  request.path = path;
+  request.type = isDir ? PATH_IS_DIR : PATH_IS_FILE;
+
+  RpcTryExcept {
+    dwError = WinutilsDeletePath(hHadoopWinutilsSvcBinding, &request, &response);
+  }
+  RpcExcept(1) {
+    ulCode = RpcExceptionCode();
+    ReportClientError(L"RpcExcept", ulCode);
+    dwError = (DWORD) ulCode;
+  }
+  RpcEndExcept;
+
+  if (ERROR_SUCCESS == dwError) {
+    *pDeleted = response->deleted;
+  }
+
+done:
+  if (rpcBindingInit) RpcBindingFree(&hHadoopWinutilsSvcBinding);
+
+  if(NULL != response) MIDL_user_free(response);
+
+  LogDebugMessage(L"RpcCall_WinutilsDeletePath: %s %d: %d %d\n",
+    path, isDir, *pDeleted, dwError);
+
+  return dwError;
+}
+
+
+DWORD RpcCall_TaskCreateAsUser(
+  LPCWSTR cwd, LPCWSTR jobName, 
+  LPCWSTR user, LPCWSTR pidFile, LPCWSTR cmdLine, 
+  HANDLE* phProcess, HANDLE* phThread, HANDLE* phStdIn, HANDLE* phStdOut, HANDLE* phStdErr) 
+{
+    DWORD       dwError = EXIT_FAILURE;
+    ULONG       ulCode;
+    DWORD       dwSelfPid = GetCurrentProcessId();
+    CREATE_PROCESS_REQUEST request;
+    CREATE_PROCESS_RESPONSE *response = NULL;
+    RPC_BINDING_HANDLE hHadoopWinutilsSvcBinding;
+    BOOL rpcBindingInit = FALSE;
+
+    dwError = PrepareRpcBindingHandle(&hHadoopWinutilsSvcBinding);
+    if (dwError) {
+      ReportClientError(L"PrepareRpcBindingHandle", dwError);
+      goto done;
+    }
+    rpcBindingInit = TRUE;
+
+    ZeroMemory(&request, sizeof(request));
+    request.cwd = cwd;
+    request.jobName = jobName;
+    request.user = user;
+    request.pidFile = pidFile;
+    request.cmdLine = cmdLine;
+
+    RpcTryExcept {
+      dwError = WinutilsCreateProcessAsUser(hHadoopWinutilsSvcBinding, dwSelfPid, &request, &response);
+    }
+    RpcExcept(1) {
+      ulCode = RpcExceptionCode();
+      ReportClientError(L"RpcExcept", ulCode);
+      dwError = (DWORD) ulCode;
+    }
+    RpcEndExcept;
+
+    if (ERROR_SUCCESS == dwError) {
+      *phProcess = response->hProcess;
+      *phThread = response->hThread;
+      *phStdIn = response->hStdIn;
+      *phStdOut = response->hStdOut;
+      *phStdErr = response->hStdErr;
+    }
+
+done:
+    if (rpcBindingInit) RpcBindingFree(&hHadoopWinutilsSvcBinding);
+  
+    if (NULL != response) {
+      MIDL_user_free(response);
+    }
+    
+    return dwError;
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/src/main/winutils/config.cpp
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/config.cpp b/hadoop-common-project/hadoop-common/src/main/winutils/config.cpp
new file mode 100644
index 0000000..1e07b7f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/config.cpp
@@ -0,0 +1,174 @@
+/**
+* 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 "winutils.h"
+#include <string.h>
+#include <stdlib.h>
+#import "msxml6.dll"
+
+#define ERROR_CHECK_HRESULT_DONE(hr, message)                               \
+  if (FAILED(hr))  {                                                        \
+    dwError = (DWORD) hr;                                                   \
+    LogDebugMessage(L"%s: %x", message, hr);                                \
+    goto done;                                                              \
+  }
+
+DWORD BuildPathRelativeToModule(
+    __in LPCWSTR relativePath, 
+    __in size_t len, 
+    __out_ecount(len) LPWSTR buffer) {
+  DWORD dwError = ERROR_SUCCESS;
+  WCHAR moduleFile[MAX_PATH];
+  WCHAR modulePath[_MAX_DIR];
+  WCHAR moduleDrive[_MAX_DRIVE];
+  DWORD size;
+  HRESULT hr = S_OK;
+  errno_t errno;
+
+  size = GetModuleFileName(NULL, moduleFile, MAX_PATH);
+  dwError = GetLastError(); // Always check due to ERROR_INSUFFICIENT_BUFFER
+  if (dwError) {
+     LogDebugMessage(L"GetModuleFileName: %x\n", dwError);
+     goto done;
+  }
+
+  errno = _wsplitpath_s(moduleFile,
+     moduleDrive, _MAX_DRIVE,
+     modulePath, _MAX_DIR,
+     NULL, 0,  // fname, not interesting
+     NULL, 0); // extenssion, not interesting
+  if (errno) {
+    LogDebugMessage(L"_wsplitpath_s: %x\n", errno);
+    dwError = ERROR_BAD_PATHNAME;
+    goto done;
+  }
+
+  hr = StringCbPrintf(buffer, len, L"%s%s%s", moduleDrive, modulePath, relativePath);
+  if (FAILED(hr)) {
+    // There is no reliable HRESULT to WIN32 mapping, use code. 
+    // see http://blogs.msdn.com/b/oldnewthing/archive/2006/11/03/942851.aspx
+    //
+    dwError = HRESULT_CODE(hr);
+    goto done;
+  }
+
+  LogDebugMessage(L"BuildPathRelativeToModule: %s (%s)\n", buffer, relativePath);
+
+done:
+  return dwError;
+}
+
+DWORD GetConfigValue(
+  __in LPCWSTR relativePath,
+  __in LPCWSTR keyName, 
+  __out size_t* len, __out_ecount(len) LPCWSTR* value) {
+
+  DWORD dwError = ERROR_SUCCESS;
+  WCHAR xmlPath[MAX_PATH];
+
+  *len = 0;
+  *value = NULL;
+
+  dwError = BuildPathRelativeToModule(
+    relativePath,
+    sizeof(xmlPath)/sizeof(WCHAR),
+    xmlPath);
+
+  if (dwError) {
+    goto done;
+  }
+
+  dwError = GetConfigValueFromXmlFile(xmlPath, keyName, len, value);
+
+done:
+  if (*len) {
+    LogDebugMessage(L"GetConfigValue:%d key:%s len:%d value:%.*s from:%s\n", dwError, keyName, *len, *len, *value, xmlPath);
+  }
+  return dwError;
+}
+
+
+DWORD GetConfigValueFromXmlFile(__in LPCWSTR xmlFile, __in LPCWSTR keyName, 
+  __out size_t* outLen, __out_ecount(len) LPCWSTR* outValue) {
+
+  DWORD dwError = ERROR_SUCCESS;
+  HRESULT hr;
+  WCHAR keyXsl[8192];
+  size_t len = 0;
+  LPWSTR value = NULL;
+  BOOL comInitialized = FALSE;
+
+  *outLen = 0;
+  *outValue = NULL;
+
+  hr = CoInitialize(NULL);
+  ERROR_CHECK_HRESULT_DONE(hr, L"CoInitialize");
+  comInitialized = TRUE;
+
+  hr = StringCbPrintf(keyXsl, sizeof(keyXsl), L"//configuration/property[name='%s']/value/text()", keyName);
+  ERROR_CHECK_HRESULT_DONE(hr, L"StringCbPrintf");
+
+  try {
+    MSXML2::IXMLDOMDocument2Ptr pDoc;
+    hr = pDoc.CreateInstance(__uuidof(MSXML2::DOMDocument60), NULL, CLSCTX_INPROC_SERVER);
+    ERROR_CHECK_HRESULT_DONE(hr, L"CreateInstance");
+
+    pDoc->async = VARIANT_FALSE;
+    pDoc->validateOnParse = VARIANT_FALSE;
+    pDoc->resolveExternals = VARIANT_FALSE;
+    
+    _variant_t file(xmlFile);
+    
+    if (VARIANT_FALSE == pDoc->load(file)) {
+      dwError = pDoc->parseError->errorCode;
+      LogDebugMessage(L"load %s failed:%d %s\n", xmlFile, dwError, 
+        static_cast<LPCWSTR>(pDoc->parseError->Getreason()));
+      goto done;
+    }
+
+    MSXML2::IXMLDOMElementPtr pRoot = pDoc->documentElement;
+    MSXML2::IXMLDOMNodePtr keyNode = pRoot->selectSingleNode(keyXsl);
+
+    if (keyNode) {
+      _bstr_t bstrValue = static_cast<_bstr_t>(keyNode->nodeValue);
+      len = bstrValue.length();
+      value = (LPWSTR) LocalAlloc(LPTR, (len+1) * sizeof(WCHAR));
+      LPCWSTR lpwszValue = static_cast<LPCWSTR>(bstrValue);
+      memcpy(value, lpwszValue, (len) * sizeof(WCHAR));
+      LogDebugMessage(L"key:%s :%.*s [%s]\n", keyName, len, value, lpwszValue);
+      *outLen = len;
+      *outValue = value;
+    }
+    else {
+      LogDebugMessage(L"node Xpath:%s not found in:%s\n", keyXsl, xmlFile);
+    }
+  } 
+  catch(_com_error errorObject) {
+    dwError = errorObject.Error();
+    LogDebugMessage(L"catch _com_error:%x %s\n", dwError, errorObject.ErrorMessage());
+    goto done;
+  }
+  
+done:
+  if (comInitialized) {
+    CoUninitialize();
+  }
+  
+  return dwError;
+}
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/src/main/winutils/hadoopwinutilsvc.idl
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/hadoopwinutilsvc.idl b/hadoop-common-project/hadoop-common/src/main/winutils/hadoopwinutilsvc.idl
new file mode 100644
index 0000000..ee3685c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/hadoopwinutilsvc.idl
@@ -0,0 +1,130 @@
+/*
+ * 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.
+ */
+
+import "oaidl.idl";
+import "ocidl.idl";
+
+[
+  uuid(0492311C-1718-4F53-A6EB-86AD7039988D),
+  version(1.0),
+  pointer_default(unique),
+  //implicit_handle(handle_t hHadoopWinutilsSvcBinding),
+  endpoint("ncalrpc:[hadoopwinutilsvc]"),
+#ifndef __midl  
+  explicit_handle
+#endif  
+]
+interface HadoopWinutilSvc
+{
+  typedef struct {
+    [string] const wchar_t* cwd;
+    [string] const wchar_t* jobName;
+    [string] const wchar_t* user;
+    [string] const wchar_t* pidFile;
+    [string] const wchar_t* cmdLine;
+  } CREATE_PROCESS_REQUEST;
+
+  typedef struct {
+    LONG_PTR hProcess;
+    LONG_PTR hThread;
+    LONG_PTR hStdIn;
+    LONG_PTR hStdOut;
+    LONG_PTR hStdErr;
+  } CREATE_PROCESS_RESPONSE;
+
+  typedef struct {
+    [string] const wchar_t* filePath;
+    [string] const wchar_t* ownerName;
+    [string] const wchar_t* groupName;
+  } CHOWN_REQUEST;
+
+  typedef struct {
+    [string] const wchar_t* filePath;
+    int mode;
+  } CHMOD_REQUEST;  
+
+  typedef struct {
+    [string] const wchar_t* filePath;
+  } MKDIR_REQUEST;  
+
+  typedef enum { MOVE_FILE = 1, COPY_FILE = 2} MOVE_COPY_OPERATION;
+
+  typedef struct {
+    MOVE_COPY_OPERATION operation;
+    [string] const wchar_t* sourcePath;
+    [string] const wchar_t* destinationPath;
+    boolean replaceExisting;
+  } MOVEFILE_REQUEST;
+
+  typedef struct {
+    [string] const wchar_t* path;
+    int desiredAccess;
+    int shareMode;
+    int creationDisposition;
+    int flags;
+  } CREATEFILE_REQUEST;
+
+  typedef struct {
+    LONG_PTR hFile;
+  } CREATEFILE_RESPONSE;
+
+  typedef enum {PATH_IS_DIR = 1, PATH_IS_FILE = 2} DELETEPATH_TYPE;
+
+  typedef struct {
+  	DELETEPATH_TYPE type;
+  	[string] const wchar_t* path;
+  } DELETEPATH_REQUEST;
+
+  typedef struct {
+  	boolean deleted;
+  } DELETEPATH_RESPONSE;
+
+  typedef struct {
+  	[string] const wchar_t* taskName;
+  } KILLTASK_REQUEST;
+
+  error_status_t WinutilsKillTask(
+    [in] KILLTASK_REQUEST *request);
+
+  error_status_t WinutilsMkDir(
+    [in] MKDIR_REQUEST *request);
+
+  error_status_t WinutilsMoveFile(
+    [in] MOVEFILE_REQUEST *request);
+
+  error_status_t WinutilsChown(
+    [in] CHOWN_REQUEST *request);
+
+  error_status_t WinutilsChmod(
+    [in] CHMOD_REQUEST *request);
+
+  error_status_t WinutilsCreateFile(
+    [in] int nmPid,
+    [in] CREATEFILE_REQUEST *request,
+    [out] CREATEFILE_RESPONSE **response);
+
+  error_status_t WinutilsDeletePath(
+  	[in] DELETEPATH_REQUEST *request,
+  	[out] DELETEPATH_RESPONSE **response);
+
+  error_status_t WinutilsCreateProcessAsUser(
+    [in] int nmPid,
+    [in] CREATE_PROCESS_REQUEST *request,
+    [out] CREATE_PROCESS_RESPONSE **response);
+    
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h b/hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h
index bae754c..77fc586 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h
+++ b/hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h
@@ -30,6 +30,11 @@
 #include <ntsecapi.h>
 #include <userenv.h>
 
+#ifdef __cplusplus
+extern "C" {
+#endif
+
+
 enum EXIT_CODE
 {
   /* Common success exit code shared among all utilities */
@@ -38,6 +43,12 @@ enum EXIT_CODE
   FAILURE = EXIT_FAILURE,
   /* Failure code indicates the user does not privilege to create symlinks */
   SYMLINK_NO_PRIVILEGE = 2,
+
+  ERROR_TASK_NOT_ALIVE = 1,
+  
+  // This exit code for killed processes is compatible with Unix, where a killed
+  // process exits with 128 + signal.  For SIGKILL, this would be 128 + 9 = 137.
+  KILLED_PROCESS_EXIT_CODE = 137,
 };
 
 
@@ -101,6 +112,8 @@ void GroupsUsage(LPCWSTR program);
 int Hardlink(__in int argc, __in_ecount(argc) wchar_t *argv[]);
 void HardlinkUsage();
 
+DWORD KillTask(PCWSTR jobObjName);
+
 int Task(__in int argc, __in_ecount(argc) wchar_t *argv[]);
 void TaskUsage();
 
@@ -167,7 +180,7 @@ void UnregisterWithLsa(__in HANDLE lsaHandle);
 
 DWORD LookupKerberosAuthenticationPackageId(__in HANDLE lsaHandle, __out ULONG * packageId);
 
-DWORD CreateLogonForUser(__in HANDLE lsaHandle,
+DWORD CreateLogonTokenForUser(__in HANDLE lsaHandle,
                          __in const char * tokenSourceName, 
                          __in const char * tokenOriginName, 
                          __in ULONG authnPkgId, 
@@ -178,3 +191,102 @@ DWORD LoadUserProfileForLogon(__in HANDLE logonHandle, __out PROFILEINFO * pi);
 
 DWORD UnloadProfileForLogon(__in HANDLE logonHandle, __in PROFILEINFO * pi);
 
+DWORD EnableImpersonatePrivileges();
+
+DWORD RunService(__in int argc, __in_ecount(argc) wchar_t *argv[]);
+void ServiceUsage();
+
+
+DWORD ChangeFileOwnerBySid(__in LPCWSTR path,
+  __in_opt PSID pNewOwnerSid, __in_opt PSID pNewGroupSid);
+
+DWORD ChownImpl(
+  __in_opt LPCWSTR userName,
+  __in_opt LPCWSTR groupName,
+  __in LPCWSTR pathName);
+
+LPCWSTR GetSystemTimeString();
+
+VOID LogDebugMessage(LPCWSTR format, ...);
+
+DWORD SplitStringIgnoreSpaceW(
+  __in size_t len, 
+  __in_ecount(len) LPCWSTR source, 
+  __in WCHAR deli, 
+  __out size_t* count, __out_ecount(count) WCHAR*** out);
+
+DWORD BuildPathRelativeToModule(
+    __in LPCWSTR relativePath, 
+    __in size_t len, 
+    __out_ecount(len) LPWSTR buffer);
+
+DWORD GetConfigValue(
+  __in LPCWSTR relativePath,
+  __in LPCWSTR keyName, 
+  __out size_t* len, 
+  __out_ecount(len) LPCWSTR* value);
+DWORD GetConfigValueFromXmlFile(
+  __in LPCWSTR xmlFile, 
+  __in LPCWSTR keyName, 
+  __out size_t* len, 
+  __out_ecount(len) LPCWSTR* value);
+
+
+DWORD BuildServiceSecurityDescriptor(
+  __in ACCESS_MASK                    accessMask,
+  __in size_t                         grantSidCount,
+  __in_ecount(grantSidCount) PSID*    pGrantSids,
+  __in size_t                         denySidCount,
+  __in_ecount(denySidCount) PSID*     pDenySids,
+  __in_opt PSID                       pOwner,
+  __out PSECURITY_DESCRIPTOR*         pSD);
+
+DWORD AddNodeManagerAndUserACEsToObject(
+  __in HANDLE hProcess,
+  __in LPWSTR user);
+
+
+DWORD GetSecureJobObjectName(
+  __in LPCWSTR      jobName,
+  __in size_t       cchSecureJobName,
+  __out_ecount(cchSecureJobName) LPWSTR secureJobName);
+
+extern const WCHAR* wsceConfigRelativePath;
+
+extern LPCWSTR NM_WSCE_ALLOWED;
+
+
+#define SVCNAME       TEXT("hadoopwinutilsvc")
+#define SVCBINDING    TEXT("ncalrpc")
+
+DWORD RpcCall_WinutilsKillTask(
+  __in LPCWSTR taskName);
+
+DWORD RpcCall_TaskCreateAsUser(
+  LPCWSTR cwd, LPCWSTR jobName, 
+  LPCWSTR user, LPCWSTR pidFile, LPCWSTR cmdLine, 
+  HANDLE* phProcess, HANDLE* phThread, HANDLE* phStdIn, HANDLE* phStdOut, HANDLE* phStdErr);
+
+DWORD RpcCall_WinutilsCreateFile(
+  __in LPCWSTR path,
+  __in DWORD desiredAccess,
+  __in DWORD shareMode,
+  __in DWORD creationDisposition,
+  __in DWORD flags,
+  __out HANDLE* hFile);
+
+DWORD RpcCall_WinutilsMoveFile(
+  __in LPCWSTR    sourcePath, 
+  __in LPCWSTR    destinationPath,
+  __in BOOL       replaceExisting);
+
+DWORD RpcCall_WinutilsDeletePath(
+  __in LPCWSTR    path,
+  __in BOOL       isDir,
+  __out BOOL*     pDeleted);
+
+#ifdef __cplusplus
+}
+#endif
+
+