소스 검색

YARN-2198. Remove the need to run NodeManager as privileged account for Windows Secure Container Executor. Contributed by Remus Rusanu

Jian He 10 년 전
부모
커밋
3b12fd6cfb
37개의 변경된 파일5442개의 추가작업 그리고 673개의 파일을 삭제
  1. 3 0
      .gitignore
  2. 5 0
      hadoop-common-project/hadoop-common/pom.xml
  3. 8 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
  4. 14 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
  5. 5 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
  6. 18 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
  7. 4 0
      hadoop-common-project/hadoop-common/src/main/native/native.vcxproj
  8. 8 3
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
  9. 569 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/yarn/server/nodemanager/windows_secure_container_executor.c
  10. 27 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/yarn/server/nodemanager/windows_secure_container_executor.h
  11. 4 126
      hadoop-common-project/hadoop-common/src/main/winutils/chown.c
  12. 498 0
      hadoop-common-project/hadoop-common/src/main/winutils/client.c
  13. 174 0
      hadoop-common-project/hadoop-common/src/main/winutils/config.cpp
  14. 130 0
      hadoop-common-project/hadoop-common/src/main/winutils/hadoopwinutilsvc.idl
  15. 113 1
      hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h
  16. 706 42
      hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c
  17. 24 66
      hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.vcxproj
  18. 23 0
      hadoop-common-project/hadoop-common/src/main/winutils/main.c
  19. 1485 0
      hadoop-common-project/hadoop-common/src/main/winutils/service.c
  20. 751 237
      hadoop-common-project/hadoop-common/src/main/winutils/task.c
  21. 64 0
      hadoop-common-project/hadoop-common/src/main/winutils/winutils.mc
  22. 4 14
      hadoop-common-project/hadoop-common/src/main/winutils/winutils.sln
  23. 36 70
      hadoop-common-project/hadoop-common/src/main/winutils/winutils.vcxproj
  24. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ProcessTree.java
  25. 3 0
      hadoop-yarn-project/CHANGES.txt
  26. 14 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
  27. 46 27
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java
  28. 4 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
  29. 606 47
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java
  30. 12 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
  31. 6 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java
  32. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
  33. 7 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDefaultContainerExecutor.java
  34. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutor.java
  35. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutorWithMocks.java
  36. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
  37. 63 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/SecureContainer.apt.vm

+ 3 - 0
.gitignore

@@ -4,6 +4,9 @@
 *.orig
 *.rej
 **/.keep
+*.sdf
+*.suo
+*.vcxproj.user
 .idea
 .svn
 .classpath

+ 5 - 0
hadoop-common-project/hadoop-common/pom.xml

@@ -34,6 +34,8 @@
     <kdc.resource.dir>src/test/resources/kdc</kdc.resource.dir>
     <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>
 
 
@@ -714,6 +716,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>

+ 8 - 1
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);

+ 14 - 4
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
@@ -406,6 +411,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
@@ -418,8 +427,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);
@@ -429,8 +439,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

+ 5 - 0
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,

+ 18 - 1
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() {
+    }
   }
   
   /**

+ 4 - 0
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">

+ 8 - 3
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];

+ 569 - 0
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);
+}
+

+ 27 - 0
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);
+
+

+ 4 - 126
hadoop-common-project/hadoop-common/src/main/winutils/chown.c

@@ -17,93 +17,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
 //
@@ -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;
 }

+ 498 - 0
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;
+}
+

+ 174 - 0
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;
+}
+
+

+ 130 - 0
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);
+    
+}

+ 113 - 1
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
+
+

+ 706 - 42
hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c

@@ -19,9 +19,24 @@
 #pragma comment(lib, "netapi32.lib")
 #pragma comment(lib, "Secur32.lib")
 #pragma comment(lib, "Userenv.lib")
+#pragma comment(lib, "Ntdsapi.lib")
+
 #include "winutils.h"
+#include <ctype.h>
+#include <Winsvc.h>
 #include <authz.h>
 #include <sddl.h>
+#include <Ntdsapi.h>
+#include <malloc.h>
+
+#define WIDEN_STRING(x) WIDEN_STRING_(x)
+#define WIDEN_STRING_(x) L ## x
+#define STRINGIFY(x) STRINGIFY_(x)
+#define STRINGIFY_(x) #x
+
+
+#pragma message("WSCE config is " STRINGIFY(WSCE_CONFIG_DIR) "\\" STRINGIFY(WSCE_CONFIG_FILE))
+const WCHAR* wsceConfigRelativePath = WIDEN_STRING(STRINGIFY(WSCE_CONFIG_DIR)) L"\\" WIDEN_STRING(STRINGIFY(WSCE_CONFIG_FILE));
 
 /*
  * The array of 12 months' three-letter abbreviations 
@@ -237,10 +252,10 @@ ConvertToLongPathExit:
 // Function: IsDirFileInfo
 //
 // Description:
-//	Test if the given file information is a directory
+//  Test if the given file information is a directory
 //
 // Returns:
-//	TRUE if it is a directory
+//  TRUE if it is a directory
 //  FALSE otherwise
 //
 // Notes:
@@ -257,10 +272,10 @@ BOOL IsDirFileInfo(const BY_HANDLE_FILE_INFORMATION *fileInformation)
 // Function: CheckFileAttributes
 //
 // Description:
-//	Check if the given file has all the given attribute(s)
+//  Check if the given file has all the given attribute(s)
 //
 // Returns:
-//	ERROR_SUCCESS on success
+//  ERROR_SUCCESS on success
 //  error code otherwise
 //
 // Notes:
@@ -281,10 +296,10 @@ static DWORD FileAttributesCheck(
 // Function: IsDirectory
 //
 // Description:
-//	Check if the given file is a directory
+//  Check if the given file is a directory
 //
 // Returns:
-//	ERROR_SUCCESS on success
+//  ERROR_SUCCESS on success
 //  error code otherwise
 //
 // Notes:
@@ -298,10 +313,10 @@ DWORD DirectoryCheck(__in LPCWSTR pathName, __out PBOOL res)
 // Function: IsReparsePoint
 //
 // Description:
-//	Check if the given file is a reparse point
+//  Check if the given file is a reparse point
 //
 // Returns:
-//	ERROR_SUCCESS on success
+//  ERROR_SUCCESS on success
 //  error code otherwise
 //
 // Notes:
@@ -315,10 +330,10 @@ static DWORD ReparsePointCheck(__in LPCWSTR pathName, __out PBOOL res)
 // Function: CheckReparseTag
 //
 // Description:
-//	Check if the given file is a reparse point of the given tag.
+//  Check if the given file is a reparse point of the given tag.
 //
 // Returns:
-//	ERROR_SUCCESS on success
+//  ERROR_SUCCESS on success
 //  error code otherwise
 //
 // Notes:
@@ -356,10 +371,10 @@ static DWORD ReparseTagCheck(__in LPCWSTR path, __in DWORD tag, __out PBOOL res)
 // Function: IsSymbolicLink
 //
 // Description:
-//	Check if the given file is a symbolic link.
+//  Check if the given file is a symbolic link.
 //
 // Returns:
-//	ERROR_SUCCESS on success
+//  ERROR_SUCCESS on success
 //  error code otherwise
 //
 // Notes:
@@ -373,10 +388,10 @@ DWORD SymbolicLinkCheck(__in LPCWSTR pathName, __out PBOOL res)
 // Function: IsJunctionPoint
 //
 // Description:
-//	Check if the given file is a junction point.
+//  Check if the given file is a junction point.
 //
 // Returns:
-//	ERROR_SUCCESS on success
+//  ERROR_SUCCESS on success
 //  error code otherwise
 //
 // Notes:
@@ -390,14 +405,14 @@ DWORD JunctionPointCheck(__in LPCWSTR pathName, __out PBOOL res)
 // Function: GetSidFromAcctNameW
 //
 // Description:
-//	To retrieve the SID for a user account
+//  To retrieve the SID for a user account
 //
 // Returns:
-//	ERROR_SUCCESS: on success
+//  ERROR_SUCCESS: on success
 //  Other error code: otherwise
 //
 // Notes:
-//	Caller needs to destroy the memory of Sid by calling LocalFree()
+//  Caller needs to destroy the memory of Sid by calling LocalFree()
 //
 DWORD GetSidFromAcctNameW(__in PCWSTR acctName, __out PSID *ppSid)
 {
@@ -479,10 +494,10 @@ DWORD GetSidFromAcctNameW(__in PCWSTR acctName, __out PSID *ppSid)
 // Function: GetUnixAccessMask
 //
 // Description:
-//	Compute the 3 bit Unix mask for the owner, group, or, others
+//  Compute the 3 bit Unix mask for the owner, group, or, others
 //
 // Returns:
-//	The 3 bit Unix mask in INT
+//  The 3 bit Unix mask in INT
 //
 // Notes:
 //
@@ -506,10 +521,10 @@ static INT GetUnixAccessMask(ACCESS_MASK Mask)
 // Function: GetAccess
 //
 // Description:
-//	Get Windows acces mask by AuthZ methods
+//  Get Windows acces mask by AuthZ methods
 //
 // Returns:
-//	ERROR_SUCCESS: on success
+//  ERROR_SUCCESS: on success
 //
 // Notes:
 //
@@ -554,10 +569,10 @@ static DWORD GetAccess(AUTHZ_CLIENT_CONTEXT_HANDLE hAuthzClient,
 // Function: GetEffectiveRightsForSid
 //
 // Description:
-//	Get Windows acces mask by AuthZ methods
+//  Get Windows acces mask by AuthZ methods
 //
 // Returns:
-//	ERROR_SUCCESS: on success
+//  ERROR_SUCCESS: on success
 //
 // Notes:
 //   We run into problems for local user accounts when using the method
@@ -714,11 +729,11 @@ CheckAccessEnd:
 // Function: FindFileOwnerAndPermissionByHandle
 //
 // Description:
-//	Find the owner, primary group and permissions of a file object given the
+//  Find the owner, primary group and permissions of a file object given the
 //  the file object handle. The function will always follow symbolic links.
 //
 // Returns:
-//	ERROR_SUCCESS: on success
+//  ERROR_SUCCESS: on success
 //  Error code otherwise
 //
 // Notes:
@@ -778,10 +793,10 @@ FindFileOwnerAndPermissionByHandleEnd:
 // Function: FindFileOwnerAndPermission
 //
 // Description:
-//	Find the owner, primary group and permissions of a file object
+//  Find the owner, primary group and permissions of a file object
 //
 // Returns:
-//	ERROR_SUCCESS: on success
+//  ERROR_SUCCESS: on success
 //  Error code otherwise
 //
 // Notes:
@@ -1207,14 +1222,14 @@ static DWORD GetWindowsDACLs(__in INT unixMask,
 
   if (winUserAccessDenyMask &&
     !AddAccessDeniedAceEx(pNewDACL, ACL_REVISION,
-    NO_PROPAGATE_INHERIT_ACE,
+    CONTAINER_INHERIT_ACE | OBJECT_INHERIT_ACE,
     winUserAccessDenyMask, pOwnerSid))
   {
     ret = GetLastError();
     goto GetWindowsDACLsEnd;
   }
   if (!AddAccessAllowedAceEx(pNewDACL, ACL_REVISION,
-    NO_PROPAGATE_INHERIT_ACE,
+    CONTAINER_INHERIT_ACE | OBJECT_INHERIT_ACE,
     winUserAccessAllowMask, pOwnerSid))
   {
     ret = GetLastError();
@@ -1222,21 +1237,21 @@ static DWORD GetWindowsDACLs(__in INT unixMask,
   }
   if (winGroupAccessDenyMask &&
     !AddAccessDeniedAceEx(pNewDACL, ACL_REVISION,
-    NO_PROPAGATE_INHERIT_ACE,
+    CONTAINER_INHERIT_ACE | OBJECT_INHERIT_ACE,
     winGroupAccessDenyMask, pGroupSid))
   {
     ret = GetLastError();
     goto GetWindowsDACLsEnd;
   }
   if (!AddAccessAllowedAceEx(pNewDACL, ACL_REVISION,
-    NO_PROPAGATE_INHERIT_ACE,
+    CONTAINER_INHERIT_ACE | OBJECT_INHERIT_ACE,
     winGroupAccessAllowMask, pGroupSid))
   {
     ret = GetLastError();
     goto GetWindowsDACLsEnd;
   }
   if (!AddAccessAllowedAceEx(pNewDACL, ACL_REVISION,
-    NO_PROPAGATE_INHERIT_ACE,
+    CONTAINER_INHERIT_ACE | OBJECT_INHERIT_ACE,
     winOtherAccessAllowMask, pEveryoneSid))
   {
     ret = GetLastError();
@@ -1440,14 +1455,14 @@ ChangeFileModeByMaskEnd:
 // Function: GetAccntNameFromSid
 //
 // Description:
-//	To retrieve an account name given the SID
+//  To retrieve an account name given the SID
 //
 // Returns:
-//	ERROR_SUCCESS: on success
+//  ERROR_SUCCESS: on success
 //  Other error code: otherwise
 //
 // Notes:
-//	Caller needs to destroy the memory of account name by calling LocalFree()
+//  Caller needs to destroy the memory of account name by calling LocalFree()
 //
 DWORD GetAccntNameFromSid(__in PSID pSid, __out PWSTR *ppAcctName)
 {
@@ -1536,10 +1551,10 @@ GetAccntNameFromSidEnd:
 // Function: GetLocalGroupsForUser
 //
 // Description:
-//	Get an array of groups for the given user.
+//  Get an array of groups for the given user.
 //
 // Returns:
-//	ERROR_SUCCESS on success
+//  ERROR_SUCCESS on success
 //  Other error code on failure
 //
 // Notes:
@@ -1631,11 +1646,12 @@ GetLocalGroupsForUserEnd:
   return ret;
 }
 
+
 //----------------------------------------------------------------------------
 // Function: EnablePrivilege
 //
 // Description:
-//	Check if the process has the given privilege. If yes, enable the privilege
+//  Check if the process has the given privilege. If yes, enable the privilege
 //  to the process's access token.
 //
 // Returns:
@@ -1672,8 +1688,8 @@ DWORD EnablePrivilege(__in LPCWSTR privilegeName)
   // As stated on MSDN, we need to use GetLastError() to check if
   // AdjustTokenPrivileges() adjusted all of the specified privileges.
   //
-  if( !AdjustTokenPrivileges(hToken, FALSE, &tp, 0, NULL, NULL) ) {
-  dwErrCode = GetLastError();
+  if (!AdjustTokenPrivileges(hToken, FALSE, &tp, 0, NULL, NULL)) {
+    dwErrCode = GetLastError();
   }
   CloseHandle(hToken);
 
@@ -1706,12 +1722,15 @@ void ReportErrorCode(LPCWSTR func, DWORD err)
     (LPWSTR)&msg, 0, NULL);
   if (len > 0)
   {
+    LogDebugMessage(L"%s error (%d): %s\n", func, err, msg);
     fwprintf(stderr, L"%s error (%d): %s\n", func, err, msg);
   }
   else
   {
+    LogDebugMessage(L"%s error code: %d.\n", func, err);
     fwprintf(stderr, L"%s error code: %d.\n", func, err);
   }
+
   if (msg != NULL) LocalFree(msg);
 }
 
@@ -1843,7 +1862,7 @@ DWORD LookupKerberosAuthenticationPackageId(__in HANDLE lsaHandle, __out ULONG *
 }
   
 //----------------------------------------------------------------------------
-// Function: CreateLogonForUser
+// Function: CreateLogonTokenForUser
 //
 // Description:
 //  Contacts the local LSA and performs a logon without credential for the 
@@ -1858,7 +1877,7 @@ DWORD LookupKerberosAuthenticationPackageId(__in HANDLE lsaHandle, __out ULONG *
 //  This call assumes that all required privileges have already been enabled (TCB etc).
 //  IMPORTANT ****  tokenOriginName must be immutable!
 //
-DWORD CreateLogonForUser(__in HANDLE lsaHandle,
+DWORD CreateLogonTokenForUser(__in HANDLE lsaHandle,
                          __in const char * tokenSourceName, 
                          __in const char * tokenOriginName, // must be immutable, will not be copied!
                          __in ULONG authnPkgId, 
@@ -2026,6 +2045,8 @@ done:
   return loadProfileStatus;
 }
 
+
+
 DWORD UnloadProfileForLogon(__in HANDLE logonHandle, __in PROFILEINFO * pi)
 {
   DWORD touchProfileStatus = ERROR_ASSERTION_FAILURE; // Failure to set status should trigger error
@@ -2046,3 +2067,646 @@ DWORD UnloadProfileForLogon(__in HANDLE logonHandle, __in PROFILEINFO * pi)
 done:
   return touchProfileStatus;
 }
+
+
+//----------------------------------------------------------------------------
+// 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.
+//
+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: GetSecureJobObjectName
+//
+// Description:
+//  Creates a job object name usable in a secure environment: adds the Golbal\
+//
+
+DWORD GetSecureJobObjectName(
+  __in LPCWSTR      jobName,
+  __in size_t       cchSecureJobName,
+  __out_ecount(cchSecureJobName) LPWSTR secureJobName) {
+
+  HRESULT hr = StringCchPrintf(secureJobName, cchSecureJobName,
+    L"Global\\%s", jobName);
+
+  if (FAILED(hr)) {
+    return HRESULT_CODE(hr);
+  }
+
+  return ERROR_SUCCESS;
+}
+
+//-----------------------------------------------------------------------------
+// Function: EnableImpersonatePrivileges
+//
+// Description:
+//  Enables the required privileges for S4U impersonation
+//
+// Returns:
+// ERROR_SUCCESS: On success
+//
+DWORD EnableImpersonatePrivileges() {
+  DWORD dwError = ERROR_SUCCESS;
+  LPCWSTR privilege = NULL;
+  int crt = 0;
+
+  LPCWSTR privileges[] = {
+    SE_IMPERSONATE_NAME,
+    SE_TCB_NAME,
+    SE_ASSIGNPRIMARYTOKEN_NAME,
+    SE_INCREASE_QUOTA_NAME,
+    SE_RESTORE_NAME,
+    SE_DEBUG_NAME,
+    SE_SECURITY_NAME,
+    };
+
+  for (crt = 0; crt < sizeof(privileges)/sizeof(LPCWSTR); ++crt) {
+    LPCWSTR privilege = privileges[crt];
+    dwError = EnablePrivilege(privilege);
+    if( dwError != ERROR_SUCCESS ) {
+      LogDebugMessage(L"Failed to enable privilege: %s\n", privilege);
+      ReportErrorCode(L"EnablePrivilege", dwError);
+      goto done;
+    }    
+  }
+
+done:
+  return dwError;
+}
+
+
+//-----------------------------------------------------------------------------
+// Function: KillTask
+//
+// Description:
+//  Kills a task via a jobobject. Outputs the
+//  appropriate information to stdout on success, or stderr on failure.
+//
+// Returns:
+// ERROR_SUCCESS: On success
+// GetLastError: otherwise
+DWORD KillTask(PCWSTR jobObjName)
+{
+  DWORD dwError = ERROR_SUCCESS;
+  
+  HANDLE jobObject = OpenJobObject(JOB_OBJECT_TERMINATE, FALSE, jobObjName);
+  if(jobObject == NULL)
+  {
+    dwError = GetLastError();
+    if(dwError == ERROR_FILE_NOT_FOUND)
+    {      
+      // job object does not exist. assume its not alive
+      dwError = ERROR_SUCCESS;
+    }
+    goto done;
+  }
+
+  if(TerminateJobObject(jobObject, KILLED_PROCESS_EXIT_CODE) == 0)
+  {
+    dwError = GetLastError();
+  }
+
+done:
+  CloseHandle(jobObject);
+  
+  return dwError;
+}
+
+DWORD ChownImpl(
+  __in_opt LPCWSTR userName,
+  __in_opt LPCWSTR groupName,
+  __in LPCWSTR pathName) {
+
+  DWORD dwError;
+
+  PSID pNewOwnerSid = NULL;
+  PSID pNewGroupSid = NULL;
+
+  if (userName != NULL)
+  {
+    dwError = GetSidFromAcctNameW(userName, &pNewOwnerSid);
+    if (dwError != ERROR_SUCCESS)
+    {
+      ReportErrorCode(L"GetSidFromAcctName", dwError);
+      fwprintf(stderr, L"Invalid user name: %s\n", userName);
+      goto done;
+    }
+  }
+
+  if (groupName != NULL)
+  {
+    dwError = GetSidFromAcctNameW(groupName, &pNewGroupSid);
+    if (dwError != ERROR_SUCCESS)
+    {
+      ReportErrorCode(L"GetSidFromAcctName", dwError);
+      fwprintf(stderr, L"Invalid group name: %s\n", groupName);
+      goto done;
+    }
+  }
+
+  if (wcslen(pathName) == 0 || wcsspn(pathName, L"/?|><:*\"") != 0)
+  {
+    fwprintf(stderr, L"Incorrect file name format: %s\n", pathName);
+    goto done;
+  }
+
+  dwError = ChangeFileOwnerBySid(pathName, pNewOwnerSid, pNewGroupSid);
+  if (dwError != ERROR_SUCCESS)
+  {
+    ReportErrorCode(L"ChangeFileOwnerBySid", dwError);
+    goto done;
+  }
+done:
+  LocalFree(pNewOwnerSid);
+  LocalFree(pNewGroupSid);
+
+  return dwError;
+}
+
+
+
+LPCWSTR GetSystemTimeString() {
+  __declspec(thread) static WCHAR buffer[1024];
+  DWORD dwError;
+  FILETIME ftime;
+  SYSTEMTIME systime;
+  LARGE_INTEGER counter, frequency;
+  int subSec;
+  double qpc;
+  HRESULT hr;
+  buffer[0] = L'\0';
+
+  // GetSystemTimePreciseAsFileTime is only available in Win8+ and our libs do not link against it
+
+  GetSystemTimeAsFileTime(&ftime);
+
+  if (!FileTimeToSystemTime(&ftime, &systime)) {
+    dwError = GetLastError();
+    LogDebugMessage(L"FileTimeToSystemTime error:%d\n", dwError);
+    goto done;
+  }
+
+  // Get the ms from QPC. GetSystemTimeAdjustment is ignored...
+  
+  QueryPerformanceCounter(&counter);
+  QueryPerformanceFrequency(&frequency);
+
+  qpc = (double) counter.QuadPart / (double) frequency.QuadPart;
+  subSec = ((qpc - (long)qpc) * 1000000);
+
+  hr = StringCbPrintf(buffer, sizeof(buffer), L"%02d:%02d:%02d.%06d", 
+    (int)systime.wHour, (int)systime.wMinute, (int)systime.wSecond, (int)subSec);
+
+  if (FAILED(hr)) {
+    LogDebugMessage(L"StringCbPrintf error:%d\n", hr);
+  }
+done:
+  return buffer;
+}
+
+
+//----------------------------------------------------------------------------
+// Function: LogDebugMessage
+//
+// Description:
+//  Sends a message to the debugger console, if one is attached
+//
+// Notes:
+//  Native debugger: windbg, ntsd, cdb, visual studio
+//
+VOID LogDebugMessage(LPCWSTR format, ...) {
+  LPWSTR buffer[8192];
+  va_list args;
+  HRESULT hr;
+
+  if (!IsDebuggerPresent()) return;
+
+  va_start(args, format);
+  hr = StringCbVPrintf(buffer, sizeof(buffer), format, args);
+  if (SUCCEEDED(hr)) {
+    OutputDebugString(buffer);
+  }
+  va_end(args);
+}
+
+//----------------------------------------------------------------------------
+// Function: SplitStringIgnoreSpaceW
+//
+// Description:
+//  splits a null-terminated string based on a delimiter
+//
+// Returns:
+//  ERROR_SUCCESS: on success
+//  error code: otherwise
+//
+// Notes:
+//  The tokes are also null-terminated
+//  Caller should use LocalFree to clear outTokens
+//
+DWORD SplitStringIgnoreSpaceW(
+  __in size_t len, 
+  __in_ecount(len) LPCWSTR source, 
+  __in WCHAR deli, 
+  __out size_t* count, 
+  __out_ecount(count) WCHAR*** outTokens) {
+  
+  size_t tokenCount = 0;
+  size_t crtSource;
+  size_t crtToken = 0;
+  WCHAR* lpwszTokenStart = NULL;
+  WCHAR* lpwszTokenEnd = NULL;
+  WCHAR* lpwszBuffer = NULL;
+  size_t tokenLength = 0;
+  size_t cchBufferLength = 0;
+  WCHAR crt;
+  WCHAR** tokens = NULL;
+  enum {BLANK, TOKEN, DELIMITER} State = BLANK;
+
+  for(crtSource = 0; crtSource < len; ++crtSource) {
+    crt = source[crtSource];
+    switch(State) {
+    case BLANK: // intentional fallthrough
+    case DELIMITER:
+      if (crt == deli) {
+        State = DELIMITER;
+      } 
+      else if (!iswspace(crt)) {
+        ++tokenCount;
+        lpwszTokenEnd = lpwszTokenStart = source + crtSource;
+        State = TOKEN;
+      }
+      else {
+        State = BLANK;
+      }
+      break;
+    case TOKEN:
+      if (crt == deli) {
+        State = DELIMITER;
+        cchBufferLength += lpwszTokenEnd - lpwszTokenStart + 2;
+      }
+      else if (!iswspace(crt)) {
+        lpwszTokenEnd = source + crtSource;
+      }
+      break;
+    }
+  }
+
+  if (State == TOKEN) {
+    cchBufferLength += lpwszTokenEnd - lpwszTokenStart + 2;
+  }
+
+  LogDebugMessage(L"counted %d [buffer:%d] tokens in %s\n", tokenCount, cchBufferLength, source);
+
+  #define COPY_CURRENT_TOKEN                                              \
+    tokenLength = lpwszTokenEnd - lpwszTokenStart + 1;                    \
+    tokens[crtToken] = lpwszBuffer;                                       \
+    memcpy(tokens[crtToken], lpwszTokenStart, tokenLength*sizeof(WCHAR)); \
+    tokens[crtToken][tokenLength] = L'\0';                                \
+    lpwszBuffer += (tokenLength+1);                                       \
+    ++crtToken;
+
+  if (tokenCount) {
+
+    // We use one contigous memory for both the pointer arrays and the data copy buffers
+    // We cannot use in-place references (zero-copy) because the function users 
+    // need null-terminated strings for the tokens
+    
+    tokens = (WCHAR**) LocalAlloc(LPTR, 
+       sizeof(WCHAR*) * tokenCount +      // for the pointers
+       sizeof(WCHAR) * cchBufferLength);  // for the data
+
+    // Data will be copied after the array
+    lpwszBuffer = (WCHAR*)(((BYTE*)tokens) + (sizeof(WCHAR*) * tokenCount));
+       
+    State = BLANK;
+
+    for(crtSource = 0; crtSource < len; ++crtSource) {
+      crt = source[crtSource];
+      switch(State) {
+      case DELIMITER: // intentional fallthrough
+      case BLANK:
+        if (crt == deli) {
+          State = DELIMITER;
+        } 
+        else if (!iswspace(crt)) {
+          lpwszTokenEnd = lpwszTokenStart = source + crtSource;
+          State = TOKEN;
+        }
+        else {
+          State = BLANK;
+        }
+        break;
+      case TOKEN:
+        if (crt == deli) {
+          COPY_CURRENT_TOKEN;
+          State = DELIMITER;
+        }
+        else if (!iswspace(crt)) {
+          lpwszTokenEnd = source + crtSource;
+        }
+        break;
+      }
+    }
+
+    // Copy out last token, if any
+    if (TOKEN == State) {
+      COPY_CURRENT_TOKEN;
+    }
+  }
+
+  *count = tokenCount;
+  *outTokens = tokens;
+
+  return ERROR_SUCCESS;
+}
+
+//----------------------------------------------------------------------------
+// Function: BuildServiceSecurityDescriptor
+//
+// Description:
+//  Builds a security descriptor for an arbitrary object
+//
+// Returns:
+//  ERROR_SUCCESS: on success
+//  error code: otherwise
+//
+// Notes:
+//  The SD is a of the self-contained flavor (offsets, not pointers)
+//  Caller should use LocalFree to clear allocated pSD
+//
+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                 dwError = ERROR_SUCCESS;
+  int                   crt  = 0;
+  int                   len = 0;
+  EXPLICIT_ACCESS*      eas = NULL;
+  LPWSTR                lpszSD = NULL;
+  ULONG                 cchSD = 0;
+  HANDLE                hToken = INVALID_HANDLE_VALUE;
+  DWORD                 dwBufferSize = 0;
+  PTOKEN_USER           pTokenUser = NULL;
+  PTOKEN_PRIMARY_GROUP  pTokenGroup = NULL;
+  PSECURITY_DESCRIPTOR  pTempSD = NULL;
+  ULONG                 cbSD = 0;
+  TRUSTEE               owner, group;
+
+  ZeroMemory(&owner, sizeof(owner));
+
+  // We'll need our own SID to add as SD owner
+  if (!OpenProcessToken(GetCurrentProcess(), TOKEN_QUERY, &hToken)) {
+    dwError = GetLastError();
+    LogDebugMessage(L"OpenProcessToken: %d\n", dwError);
+    goto done;  
+  }
+
+  if (NULL == pOwner) {
+    if (!GetTokenInformation(hToken, TokenUser, NULL, 0, &dwBufferSize)) {
+      dwError = GetLastError();
+      if (ERROR_INSUFFICIENT_BUFFER != dwError) {
+        LogDebugMessage(L"GetTokenInformation: %d\n", dwError);
+        goto done;
+      }
+    }
+
+    pTokenUser = (PTOKEN_USER) LocalAlloc(LPTR, dwBufferSize);
+    if (NULL == pTokenUser) {
+      dwError = GetLastError();
+      LogDebugMessage(L"LocalAlloc:pTokenUser: %d\n", dwError);
+      goto done; 
+    }
+
+    if (!GetTokenInformation(hToken, TokenUser, pTokenUser, dwBufferSize, &dwBufferSize)) {
+      dwError = GetLastError();
+      LogDebugMessage(L"GetTokenInformation: %d\n", dwError);
+      goto done; 
+    }
+
+    if (!IsValidSid(pTokenUser->User.Sid)) {
+      dwError = ERROR_INVALID_PARAMETER;
+      LogDebugMessage(L"IsValidSid: %d\n", dwError);
+      goto done;
+    }
+    pOwner = pTokenUser->User.Sid;
+  }
+
+  dwBufferSize = 0;
+  if (!GetTokenInformation(hToken, TokenPrimaryGroup, NULL, 0, &dwBufferSize)) {
+    dwError = GetLastError();
+    if (ERROR_INSUFFICIENT_BUFFER != dwError) {
+      LogDebugMessage(L"GetTokenInformation: %d\n", dwError);
+      goto done;
+    }
+  }
+
+  pTokenGroup = (PTOKEN_USER) LocalAlloc(LPTR, dwBufferSize);
+  if (NULL == pTokenGroup) {
+    dwError = GetLastError();
+    LogDebugMessage(L"LocalAlloc:pTokenGroup: %d\n", dwError);
+    goto done; 
+  }
+
+  if (!GetTokenInformation(hToken, TokenPrimaryGroup, pTokenGroup, dwBufferSize, &dwBufferSize)) {
+    dwError = GetLastError();
+    LogDebugMessage(L"GetTokenInformation: %d\n", dwError);
+    goto done; 
+  }
+
+  if (!IsValidSid(pTokenGroup->PrimaryGroup)) {
+    dwError = ERROR_INVALID_PARAMETER;
+    LogDebugMessage(L"IsValidSid: %d\n", dwError);
+    goto done;
+  }  
+
+  owner.TrusteeForm = TRUSTEE_IS_SID;
+  owner.TrusteeType = TRUSTEE_IS_UNKNOWN;
+  owner.ptstrName = (LPCWSTR) pOwner;
+
+  group.TrusteeForm = TRUSTEE_IS_SID;
+  group.TrusteeType = TRUSTEE_IS_UNKNOWN;
+  group.ptstrName = (LPCWSTR) pTokenGroup->PrimaryGroup;
+
+  eas = (EXPLICIT_ACCESS*) LocalAlloc(LPTR, sizeof(EXPLICIT_ACCESS) * (grantSidCount + denySidCount));
+  if (NULL == eas) {
+    dwError = ERROR_OUTOFMEMORY;
+    LogDebugMessage(L"LocalAlloc: %d\n", dwError);
+    goto done;
+  }
+
+  // Build the granted list
+  for (crt = 0; crt < grantSidCount; ++crt) {
+    eas[crt].grfAccessPermissions = accessMask;
+    eas[crt].grfAccessMode = GRANT_ACCESS;
+    eas[crt].grfInheritance = NO_INHERITANCE;
+    eas[crt].Trustee.TrusteeForm = TRUSTEE_IS_SID;
+    eas[crt].Trustee.TrusteeType = TRUSTEE_IS_UNKNOWN;
+    eas[crt].Trustee.ptstrName = (LPCWSTR) pGrantSids[crt];
+    eas[crt].Trustee.pMultipleTrustee = NULL;
+    eas[crt].Trustee.MultipleTrusteeOperation = NO_MULTIPLE_TRUSTEE;
+  }
+
+  // Build the deny list
+  for (; crt < grantSidCount + denySidCount; ++crt) {
+    eas[crt].grfAccessPermissions = accessMask;
+    eas[crt].grfAccessMode = DENY_ACCESS;
+    eas[crt].grfInheritance = NO_INHERITANCE;
+    eas[crt].Trustee.TrusteeForm = TRUSTEE_IS_SID;
+    eas[crt].Trustee.TrusteeType = TRUSTEE_IS_UNKNOWN;
+    eas[crt].Trustee.ptstrName = (LPCWSTR) pDenySids[crt - grantSidCount];
+    eas[crt].Trustee.pMultipleTrustee = NULL;
+    eas[crt].Trustee.MultipleTrusteeOperation = NO_MULTIPLE_TRUSTEE;
+  }
+
+  dwError = BuildSecurityDescriptor(
+    &owner,
+    &group,
+    crt,
+    eas,
+    0,    // cCountOfAuditEntries
+    NULL, // pListOfAuditEntries
+    NULL, // pOldSD
+    &cbSD, 
+    &pTempSD);
+  if (ERROR_SUCCESS != dwError) {
+    LogDebugMessage(L"BuildSecurityDescriptor: %d\n", dwError);
+    goto done;
+  }
+  
+  *pSD = pTempSD;
+  pTempSD = NULL;
+
+  if (IsDebuggerPresent()) {
+    ConvertSecurityDescriptorToStringSecurityDescriptor(*pSD, 
+      SDDL_REVISION_1,
+      DACL_SECURITY_INFORMATION,
+      &lpszSD,
+      &cchSD);
+    LogDebugMessage(L"pSD: %.*s\n", cchSD, lpszSD);
+  }
+  
+done:
+  if (eas) LocalFree(eas);
+  if (pTokenUser) LocalFree(pTokenUser);
+  if (INVALID_HANDLE_VALUE != hToken) CloseHandle(hToken);
+  if (lpszSD) LocalFree(lpszSD);
+  if (pTempSD) LocalFree(pTempSD);
+  return dwError;
+}
+
+//----------------------------------------------------------------------------
+// Function: MIDL_user_allocate
+//
+// Description:
+//  Hard-coded function name used by RPC midl code for allocations
+//
+// Notes:
+//  Must match the de-allocation mechanism used in MIDL_user_free
+//
+void __RPC_FAR * __RPC_USER MIDL_user_allocate(size_t len)
+{
+    return LocalAlloc(LPTR, len);
+}
+ 
+ //----------------------------------------------------------------------------
+ // Function: MIDL_user_free
+ //
+ // Description:
+ //  Hard-coded function name used by RPC midl code for deallocations
+ //
+ // NoteS:
+ //  Must match the allocation mechanism used in MIDL_user_allocate
+ //
+void __RPC_USER MIDL_user_free(void __RPC_FAR * ptr)
+{
+    LocalFree(ptr);
+}
+

+ 24 - 66
hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.vcxproj

@@ -19,18 +19,10 @@
 
 <Project DefaultTargets="Build" ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
   <ItemGroup Label="ProjectConfigurations">
-    <ProjectConfiguration Include="Debug|Win32">
-      <Configuration>Debug</Configuration>
-      <Platform>Win32</Platform>
-    </ProjectConfiguration>
     <ProjectConfiguration Include="Debug|x64">
       <Configuration>Debug</Configuration>
       <Platform>x64</Platform>
     </ProjectConfiguration>
-    <ProjectConfiguration Include="Release|Win32">
-      <Configuration>Release</Configuration>
-      <Platform>Win32</Platform>
-    </ProjectConfiguration>
     <ProjectConfiguration Include="Release|x64">
       <Configuration>Release</Configuration>
       <Platform>x64</Platform>
@@ -42,22 +34,11 @@
     <RootNamespace>winutils</RootNamespace>
   </PropertyGroup>
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.Default.props" />
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'" Label="Configuration">
-    <ConfigurationType>StaticLibrary</ConfigurationType>
-    <UseDebugLibraries>true</UseDebugLibraries>
-    <CharacterSet>Unicode</CharacterSet>
-  </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'" Label="Configuration">
     <ConfigurationType>StaticLibrary</ConfigurationType>
     <UseDebugLibraries>true</UseDebugLibraries>
     <CharacterSet>Unicode</CharacterSet>
   </PropertyGroup>
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="Configuration">
-    <ConfigurationType>StaticLibrary</ConfigurationType>
-    <UseDebugLibraries>false</UseDebugLibraries>
-    <WholeProgramOptimization>true</WholeProgramOptimization>
-    <CharacterSet>Unicode</CharacterSet>
-  </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="Configuration">
     <ConfigurationType>StaticLibrary</ConfigurationType>
     <UseDebugLibraries>false</UseDebugLibraries>
@@ -67,15 +48,9 @@
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.props" />
   <ImportGroup Label="ExtensionSettings">
   </ImportGroup>
-  <ImportGroup Label="PropertySheets" Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
-    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
-  </ImportGroup>
   <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'" Label="PropertySheets">
     <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
   </ImportGroup>
-  <ImportGroup Label="PropertySheets" Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
-    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
-  </ImportGroup>
   <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="PropertySheets">
     <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
   </ImportGroup>
@@ -83,57 +58,35 @@
   <PropertyGroup>
     <IncludePath>include;$(IncludePath)</IncludePath>
   </PropertyGroup>
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
-    <LinkIncremental>true</LinkIncremental>
-  </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'">
     <LinkIncremental>true</LinkIncremental>
-    <OutDir />
-    <IntDir>..\..\..\target\winutils\$(Configuration)\</IntDir>
-  </PropertyGroup>
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
-    <LinkIncremental>false</LinkIncremental>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
     <LinkIncremental>false</LinkIncremental>
-    <OutDir>..\..\..\target\bin\</OutDir>
-    <IntDir>..\..\..\target\winutils\$(Platform)\$(Configuration)\</IntDir>
   </PropertyGroup>
-  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
-    <ClCompile>
-      <PrecompiledHeader>
-      </PrecompiledHeader>
-      <WarningLevel>Level3</WarningLevel>
-      <Optimization>Disabled</Optimization>
-      <PreprocessorDefinitions>WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions)</PreprocessorDefinitions>
-    </ClCompile>
-    <Link>
-      <SubSystem>Console</SubSystem>
-      <GenerateDebugInformation>true</GenerateDebugInformation>
-    </Link>
-  </ItemDefinitionGroup>
   <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'">
     <ClCompile>
       <PrecompiledHeader>
       </PrecompiledHeader>
       <WarningLevel>Level4</WarningLevel>
       <Optimization>Disabled</Optimization>
-      <PreprocessorDefinitions>WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <PreprocessorDefinitions>WIN32;_DEBUG;_UNICODE;UNICODE;WSCE_CONFIG_DIR=$(WsceConfigDir);WSCE_CONFIG_FILE=$(WsceConfigFile);%(PreprocessorDefinitions)</PreprocessorDefinitions>
     </ClCompile>
     <Link>
       <SubSystem>Console</SubSystem>
       <GenerateDebugInformation>true</GenerateDebugInformation>
     </Link>
   </ItemDefinitionGroup>
-  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
     <ClCompile>
       <WarningLevel>Level3</WarningLevel>
       <PrecompiledHeader>
       </PrecompiledHeader>
-      <Optimization>MaxSpeed</Optimization>
+      <!-- <Optimization>MaxSpeed</Optimization> -->
+      <Optimization>Disabled</Optimization>
       <FunctionLevelLinking>true</FunctionLevelLinking>
       <IntrinsicFunctions>true</IntrinsicFunctions>
-      <PreprocessorDefinitions>WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <PreprocessorDefinitions>WIN32;NDEBUG;_UNICODE;UNICODE;WSCE_CONFIG_DIR=$(WsceConfigDir);WSCE_CONFIG_FILE=$(WsceConfigFile);%(PreprocessorDefinitions)</PreprocessorDefinitions>
     </ClCompile>
     <Link>
       <SubSystem>Console</SubSystem>
@@ -142,29 +95,34 @@
       <OptimizeReferences>true</OptimizeReferences>
     </Link>
   </ItemDefinitionGroup>
-  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
+  <ItemDefinitionGroup>
     <ClCompile>
-      <WarningLevel>Level3</WarningLevel>
-      <PrecompiledHeader>
-      </PrecompiledHeader>
-      <Optimization>MaxSpeed</Optimization>
-      <FunctionLevelLinking>true</FunctionLevelLinking>
-      <IntrinsicFunctions>true</IntrinsicFunctions>
-      <PreprocessorDefinitions>WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <AdditionalIncludeDirectories>$(IntermediateOutputPath)</AdditionalIncludeDirectories>
     </ClCompile>
-    <Link>
-      <SubSystem>Console</SubSystem>
-      <GenerateDebugInformation>true</GenerateDebugInformation>
-      <EnableCOMDATFolding>true</EnableCOMDATFolding>
-      <OptimizeReferences>true</OptimizeReferences>
-    </Link>
+    <Midl>
+      <ApplicationConfigurationMode>true</ApplicationConfigurationMode>
+      <TargetEnvironment>X64</TargetEnvironment>
+      <OutputDirectory>$(IntermediateOutputPath)</OutputDirectory>
+      <GenerateStublessProxies>true</GenerateStublessProxies>
+      <ValidateAllParameters>true</ValidateAllParameters>
+      <WarnAsError>true</WarnAsError>
+      <WarningLevel>2</WarningLevel>
+    </Midl>
   </ItemDefinitionGroup>
   <ItemGroup>
+    <ClCompile Include="client.c" />
+    <ClCompile Include="$(IntermediateOutputPath)\hadoopwinutilsvc_c.c" />
     <ClCompile Include="libwinutils.c" />
   </ItemGroup>
+  <ItemGroup>
+    <ClCompile Include="config.cpp" />
+  </ItemGroup>
   <ItemGroup>
     <ClInclude Include="include/winutils.h" />
   </ItemGroup>
+  <ItemGroup>
+    <Midl Include="hadoopwinutilsvc.idl" />
+  </ItemGroup>
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />
   <ImportGroup Label="ExtensionTargets">
   </ImportGroup>

+ 23 - 0
hadoop-common-project/hadoop-common/src/main/winutils/main.c

@@ -17,12 +17,27 @@
 
 #include "winutils.h"
 
+#include <winbase.h>
+
 static void Usage(LPCWSTR program);
 
+LONG WINAPI WinutilsSehUnhandled(_In_  struct _EXCEPTION_POINTERS *ecxr) {
+	LogDebugMessage(L"unhandled SEH: code:%x flags:%d\n", 
+		ecxr->ExceptionRecord->ExceptionCode,
+		ecxr->ExceptionRecord->ExceptionFlags);
+  fwprintf(stderr, L"Unhandled exception code:%x at address:%p",
+		ecxr->ExceptionRecord->ExceptionCode,
+		ecxr->ExceptionRecord->ExceptionAddress);
+	ExitProcess(ERROR_UNHANDLED_EXCEPTION);
+	return EXCEPTION_EXECUTE_HANDLER; // not that it matters...
+}
+
 int wmain(__in int argc, __in_ecount(argc) wchar_t* argv[])
 {
   LPCWSTR cmd = NULL;
 
+  SetUnhandledExceptionFilter(WinutilsSehUnhandled);
+
   if (argc < 2)
   {
     Usage(argv[0]);
@@ -67,6 +82,10 @@ int wmain(__in int argc, __in_ecount(argc) wchar_t* argv[])
   {
     return SystemInfo();
   }
+  else if (wcscmp(L"service", cmd) == 0)
+  {
+    return RunService(argc - 1, argv + 1);
+  }
   else if (wcscmp(L"help", cmd) == 0)
   {
     Usage(argv[0]);
@@ -119,5 +138,9 @@ The available commands and their usages are:\n\n", program);
 
   fwprintf(stdout, L"%-15s%s\n\n", L"task", L"Task operations.");
   TaskUsage();
+
+  fwprintf(stdout, L"%-15s%s\n\n", L"service", L"Service operations.");
+  ServiceUsage();
+
   fwprintf(stdout, L"\n\n");
 }

+ 1485 - 0
hadoop-common-project/hadoop-common/src/main/winutils/service.c

@@ -0,0 +1,1485 @@
+/**
+* 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 "winutils_msg.h"
+#include <Winsvc.h>
+#include <errno.h>
+#include <malloc.h>
+#include <strsafe.h>
+#include <authz.h>
+#include <sddl.h>
+#include "hadoopwinutilsvc_h.h"
+
+#pragma comment(lib, "Rpcrt4.lib")
+#pragma comment(lib, "advapi32.lib")
+#pragma comment(lib, "authz.lib")
+
+LPCWSTR NM_WSCE_ALLOWED     = L"yarn.nodemanager.windows-secure-container-executor.allowed";
+LPCWSTR NM_WSCE_JOB_NAME    = L"yarn.nodemanager.windows-secure-container-executor.job-name";
+LPCWSTR NM_WSCE_LOCAL_DIRS  = L"yarn.nodemanager.windows-secure-container-executor.local-dirs";
+
+#define SERVICE_ACCESS_MASK 0x00000001
+
+SERVICE_STATUS          gSvcStatus;
+SERVICE_STATUS_HANDLE   gSvcStatusHandle;
+HANDLE                  ghSvcStopEvent = INVALID_HANDLE_VALUE;
+HANDLE                  ghWaitObject = INVALID_HANDLE_VALUE;
+HANDLE                  ghEventLog = INVALID_HANDLE_VALUE;
+BOOL                    isListenning = FALSE;
+PSECURITY_DESCRIPTOR    pAllowedSD = NULL;
+LPWSTR*                 gLocalDirs = NULL;
+size_t                  gLocalDirsCount = 0;
+int*                    gCchLocalDir = NULL;
+LPCWSTR                 gJobName = NULL;
+
+VOID SvcError(DWORD dwError);
+VOID WINAPI SvcMain(DWORD dwArg, LPTSTR* lpszArgv);
+DWORD SvcInit();
+DWORD RpcInit();
+DWORD AuthInit();
+VOID ReportSvcStatus( DWORD dwCurrentState,
+                      DWORD dwWin32ExitCode,
+                      DWORD dwWaitHint);
+VOID WINAPI SvcCtrlHandler( DWORD dwCtrl );
+VOID CALLBACK SvcShutdown(
+  _In_  PVOID lpParameter,
+  _In_  BOOLEAN TimerOrWaitFired);
+
+#define CHECK_ERROR_DONE(status, expected, category, message)       \
+  if (status != expected) {                                         \
+    ReportSvcCheckError(                                            \
+      EVENTLOG_ERROR_TYPE,                                          \
+      category,                                                     \
+      status,                                                       \
+      message);                                                     \
+    goto done;                                                      \
+  } else {                                                          \
+    LogDebugMessage(L"%s: OK\n", message);                          \
+  }
+
+
+#define CHECK_RPC_STATUS_DONE(status, message)                      \
+ CHECK_ERROR_DONE(status, RPC_S_OK, SERVICE_CATEGORY, message)
+
+#define CHECK_SVC_STATUS_DONE(status, message)                      \
+ CHECK_ERROR_DONE(status, ERROR_SUCCESS, SERVICE_CATEGORY, message)
+
+#define CHECK_UNWIND_RPC(rpcCall) {                                 \
+    unwindStatus = rpcCall;                                         \
+    if (RPC_S_OK != unwindStatus) {                                 \
+      ReportSvcCheckError(                                          \
+          EVENTLOG_WARNING_TYPE,                                    \
+          SERVICE_CATEGORY,                                         \
+          unwindStatus,                                             \
+          L#rpcCall);                                               \
+      }                                                             \
+    }
+
+//----------------------------------------------------------------------------
+// Function: ReportSvcCheckError
+//
+// Description:
+//  Reports an error with the system event log and to debugger console (if present)
+//
+void ReportSvcCheckError(WORD type, WORD category, DWORD dwError, LPCWSTR message) {
+    int       len;
+    LPWSTR    systemMsg = NULL;
+    LPWSTR    appMsg = NULL;
+    DWORD     dwReportError;
+    LPWSTR    reportMsg = NULL;
+    WCHAR     hexError[32];
+    LPCWSTR   inserts[] = {message, NULL, NULL, NULL};
+    HRESULT   hr;
+
+    hr = StringCbPrintf(hexError, sizeof(hexError), TEXT("%x"), dwError);
+    if (SUCCEEDED(hr)) {
+      inserts[1] = hexError;
+    }
+    else {
+      inserts[1] = L"(Failed to format dwError as string)";
+    }
+    
+    len = FormatMessageW(
+      FORMAT_MESSAGE_ALLOCATE_BUFFER | FORMAT_MESSAGE_FROM_SYSTEM,
+      NULL, dwError,
+      MAKELANGID(LANG_NEUTRAL, SUBLANG_DEFAULT),
+      (LPWSTR)&systemMsg, 0, NULL);
+  
+    if (len) {
+      inserts[2] = systemMsg;
+    }
+    else {
+      inserts[2] = L"(Failed to get the system error message)";
+    }
+
+    LogDebugMessage(L"%s:%d %.*s\n", message, dwError, len, systemMsg);
+  
+    if (INVALID_HANDLE_VALUE != ghEventLog) {
+      if (!ReportEvent(ghEventLog, type, category, MSG_CHECK_ERROR,
+        NULL,         // lpUserSid
+        (WORD) 3,     // wNumStrings
+        (DWORD) 0,    // dwDataSize
+        inserts,      // *lpStrings
+        NULL          // lpRawData
+        )) {
+          // We tried to report and failed. Send to dbg.
+          dwReportError = GetLastError();
+          len = FormatMessageW(
+            FORMAT_MESSAGE_ALLOCATE_BUFFER | FORMAT_MESSAGE_FROM_SYSTEM,
+            NULL, dwReportError,
+            MAKELANGID(LANG_NEUTRAL, SUBLANG_DEFAULT),
+            (LPWSTR)&reportMsg, 0, NULL);
+          LogDebugMessage(L"ReportEvent: Error:%d %.*s\n", dwReportError, reportMsg);
+      }
+    };
+    
+    if (NULL != systemMsg) LocalFree(systemMsg);
+    if (NULL != reportMsg) LocalFree(reportMsg);
+}
+
+
+VOID ReportSvcMessage(WORD type, WORD category, DWORD msgId) {
+  DWORD dwError;
+  
+  if (INVALID_HANDLE_VALUE != ghEventLog) {
+    if (!ReportEvent(ghEventLog, type, category, msgId,
+      NULL,         // lpUserSid
+      (WORD) 0,     // wNumStrings
+      (DWORD) 0,    // dwDataSize
+      NULL,         // *lpStrings
+      NULL          // lpRawData
+      )) {
+        // We tried to report and failed but debugger is attached. Send to dbg.
+        dwError = GetLastError();
+        LogDebugMessage(L"ReportEvent: error %d\n", dwError);
+    }
+  }
+}
+
+//----------------------------------------------------------------------------
+// Function: IsSidInList
+//
+// Description:
+//  Finds a SID in an array of SID*
+//
+BOOL IsSidInList(
+  __in PSID trustee, 
+  __in size_t cAllowedSids, 
+  __in_ecount(cAllowedSids) PSID* allowedSids) {
+
+  int crtSid = 0;
+  
+  for (crtSid = 0; crtSid < cAllowedSids; ++crtSid) {
+    if (EqualSid(trustee, allowedSids[crtSid])) {
+      return TRUE;
+    }
+  }
+  return FALSE;
+}
+
+
+//----------------------------------------------------------------------------
+// Function: InitLocalDirs
+//
+// Description:
+//  Validates that the wsceConfigRelativePath file is only writable by Administrators
+//
+DWORD ValidateConfigurationFile() {
+  DWORD dwError = ERROR_SUCCESS;
+  WCHAR xmlPath[MAX_PATH];
+  PSECURITY_DESCRIPTOR pSd = NULL;
+  BOOL daclPresent = FALSE;
+  BOOL daclDefaulted = FALSE;
+  PACL pDacl = NULL;
+  int crt = 0, crtSid = 0;
+  WELL_KNOWN_SID_TYPE allowedSidTypes[] = {
+    WinLocalSystemSid,
+    WinBuiltinAdministratorsSid};
+  ACL_SIZE_INFORMATION aclInfo;
+  DWORD cbSid = SECURITY_MAX_SID_SIZE;
+  PSID* allowedSids = NULL; 
+  int cAllowedSids = 0;
+  BOOL isSidDefaulted;
+  PSID sidOwner = NULL;
+  PSID sidGroup = NULL;
+
+  allowedSids = (PSID*) LocalAlloc(
+    LPTR, 
+    sizeof(PSID) * sizeof(allowedSidTypes) / sizeof(WELL_KNOWN_SID_TYPE));
+  if (NULL == allowedSids) {
+    dwError = ERROR_OUTOFMEMORY;
+    CHECK_SVC_STATUS_DONE(dwError, L"LocalAlloc");
+  }
+
+  for(crt = 0; crt < sizeof(allowedSidTypes) / sizeof(WELL_KNOWN_SID_TYPE); ++crt) {
+    allowedSids[crt] = LocalAlloc(LPTR, SECURITY_MAX_SID_SIZE);
+    if (NULL == allowedSids[crt]) {
+      dwError = ERROR_OUTOFMEMORY;
+      CHECK_SVC_STATUS_DONE(dwError, L"LocalAlloc");
+    }
+
+    cbSid = SECURITY_MAX_SID_SIZE;
+    
+    if (!CreateWellKnownSid(
+      allowedSidTypes[crt], NULL, allowedSids[crt], &cbSid)) {
+      dwError = GetLastError();
+      CHECK_SVC_STATUS_DONE(dwError, L"CreateWellKnownSid");
+    }
+    ++cAllowedSids;
+  }
+
+  dwError = BuildPathRelativeToModule(
+    wsceConfigRelativePath,
+    sizeof(xmlPath)/sizeof(WCHAR),
+    xmlPath);
+  CHECK_SVC_STATUS_DONE(dwError, L"BuildPathRelativeToModule");
+
+  dwError = GetNamedSecurityInfo(
+    xmlPath, 
+    SE_FILE_OBJECT,
+    DACL_SECURITY_INFORMATION,
+    NULL, NULL, NULL, NULL, &pSd);
+  CHECK_SVC_STATUS_DONE(dwError, L"GetNamedSecurityInfo");
+
+  if (!GetSecurityDescriptorDacl(
+    pSd,
+    &daclPresent,
+    &pDacl,
+    &daclDefaulted)) {
+    dwError = GetLastError();
+    CHECK_SVC_STATUS_DONE(dwError, L"GetSecurityDescriptorDacl");
+  }
+    
+  if (!pDacl) {
+    dwError = ERROR_BAD_CONFIGURATION;
+    CHECK_SVC_STATUS_DONE(dwError, L"pDacl");
+  }
+
+  ZeroMemory(&aclInfo, sizeof(aclInfo));
+  if (!GetAclInformation(pDacl, &aclInfo, sizeof(aclInfo), AclSizeInformation)) {
+    dwError = GetLastError();
+    CHECK_SVC_STATUS_DONE(dwError, L"GetAclInformation");
+  }
+
+  // Inspect all ACEs in the file DACL.
+  // Look at all WRITE GRANTs. Make sure the trustee Sid is one of the approved Sid
+  //
+  for(crt = 0; crt < aclInfo.AceCount; ++crt) {
+
+    ACE_HEADER* aceHdr = NULL;
+    if (!GetAce(pDacl, crt, &aceHdr)) {
+      dwError = GetLastError();
+      CHECK_SVC_STATUS_DONE(dwError, L"GetAce");
+    }
+    
+    if (ACCESS_ALLOWED_ACE_TYPE == aceHdr->AceType) {
+      ACCESS_ALLOWED_ACE* pAce = (ACCESS_ALLOWED_ACE*) aceHdr;
+      if (WinMasks[WIN_WRITE] & pAce->Mask) {
+         if (!IsSidInList((PSID) &pAce->SidStart, cAllowedSids, allowedSids)) {
+            dwError = ERROR_BAD_CONFIGURATION;
+            CHECK_SVC_STATUS_DONE(dwError, L"!validSidFound");
+         }         
+      }
+    }
+  }
+  
+done:
+  if (pSd) LocalFree(pSd);
+
+  if (allowedSids) {
+    while (cAllowedSids) {
+      LocalFree(allowedSids[cAllowedSids--]);
+      }
+    LocalFree(allowedSids);
+    }
+  
+  return dwError;
+}
+
+//----------------------------------------------------------------------------
+// Function: InitJobName
+//
+// Description:
+//  Loads the job name to be used for created processes
+//
+DWORD InitJobName() {
+  DWORD     dwError = ERROR_SUCCESS;
+  size_t    len = 0;
+  LPCWSTR   value = NULL;
+  int       crt = 0;
+
+  // Services can be restarted
+  if (gJobName) LocalFree(gJobName);
+  gJobName = NULL;
+    
+  dwError = GetConfigValue(
+    wsceConfigRelativePath,
+    NM_WSCE_JOB_NAME, &len, &value);
+  CHECK_SVC_STATUS_DONE(dwError, L"GetConfigValue");
+
+  if (len) {
+    gJobName = value;
+  }
+done:
+  return dwError;
+}
+
+
+//----------------------------------------------------------------------------
+// Function: InitLocalDirs
+//
+// Description:
+//  Loads the configured local dirs
+//
+DWORD InitLocalDirs() {
+  DWORD     dwError = ERROR_SUCCESS;
+  size_t    len = 0;
+  LPCWSTR   value = NULL;
+  int       crt = 0;
+    
+
+  dwError = GetConfigValue(
+    wsceConfigRelativePath,
+    NM_WSCE_LOCAL_DIRS, &len, &value);
+  CHECK_SVC_STATUS_DONE(dwError, L"GetConfigValue");
+
+  if (0 == len) {
+    dwError = ERROR_BAD_CONFIGURATION;
+    CHECK_SVC_STATUS_DONE(dwError, NM_WSCE_LOCAL_DIRS);
+  }
+  
+  dwError = SplitStringIgnoreSpaceW(len, value, L',', &gLocalDirsCount, &gLocalDirs);
+  CHECK_SVC_STATUS_DONE(dwError, L"SplitStringIgnoreSpaceW");
+
+  if (0 == gLocalDirsCount) {
+    dwError = ERROR_BAD_CONFIGURATION;
+    CHECK_SVC_STATUS_DONE(dwError, NM_WSCE_LOCAL_DIRS);
+  }
+
+  gCchLocalDir = (int*) LocalAlloc(LPTR, sizeof(int) * gLocalDirsCount);
+  if (NULL == gCchLocalDir) {
+    dwError = ERROR_OUTOFMEMORY;
+    CHECK_SVC_STATUS_DONE(dwError, L"LocalAlloc");
+  }
+
+  for (crt = 0; crt < gLocalDirsCount; ++crt) {
+    gCchLocalDir[crt] = (int) wcsnlen(gLocalDirs[crt], MAX_PATH);
+  }
+
+done:
+  if (value) LocalFree(value);
+  
+  return dwError;
+}
+
+//----------------------------------------------------------------------------
+// Function: ValidateLocalPath
+//
+// Description:
+//  Validates that a path is within the contained local dirs
+//
+DWORD ValidateLocalPath(LPCWSTR lpszPath) {
+  DWORD   dwError = ERROR_SUCCESS;
+  int     compareResult = 0;
+  int     crt = 0;
+  int     cchLocalBuffer = 0;
+  WCHAR   localBuffer[MAX_PATH+1];
+  BOOLEAN nullFound = FALSE;
+
+  // Make a copy of the path and replace / with \ in the process
+  while(crt < MAX_PATH && !nullFound) {
+    switch(lpszPath[crt]) {
+    case L'/':
+      localBuffer[crt] = L'\\';
+      ++crt;
+      break;
+    case L'\0':
+      // NULL terminator
+      nullFound = TRUE;
+      break;
+    default:
+      localBuffer[crt] = lpszPath[crt];
+      ++crt;
+      break;
+    }
+  }
+
+  if (FALSE == nullFound) {
+    dwError = ERROR_BUFFER_OVERFLOW;
+    CHECK_SVC_STATUS_DONE(dwError, L"localBuffer");
+  }
+  
+  localBuffer[crt] = 0;
+  cchLocalBuffer = crt;
+
+  for(crt = 0; crt < gLocalDirsCount; ++crt) {
+
+    // use max len gCchLocalDir[crt] to see if it starts with this local dir
+    compareResult = CompareStringEx(
+      LOCALE_NAME_INVARIANT,
+      NORM_IGNORECASE,
+      localBuffer, gCchLocalDir[crt] <= cchLocalBuffer ? gCchLocalDir[crt] : cchLocalBuffer, 
+      gLocalDirs[crt], gCchLocalDir[crt],
+      NULL, // lpVersionInformation
+      NULL, // lpReserved
+      NULL); // lParam
+    
+    if (0 == compareResult) {
+      dwError = GetLastError();
+      CHECK_SVC_STATUS_DONE(dwError, L"CompareStringEx");
+    }
+    
+    if (CSTR_EQUAL == compareResult) {
+      break;
+    }
+  }
+
+  if (CSTR_EQUAL != compareResult) {
+    LogDebugMessage(L"ValidateLocalPath bad path: %s\n", lpszPath);
+    dwError = ERROR_BAD_PATHNAME;
+  }
+  
+done:
+  return dwError;
+}
+
+
+
+//----------------------------------------------------------------------------
+// Function: RunService
+//
+// Description:
+//  Registers with NT SCM and starts the service
+//
+// Returns:
+// ERROR_SUCCESS: On success
+// Error code otherwise: otherwise
+DWORD RunService(__in int argc, __in_ecount(argc) wchar_t *argv[])
+{
+  DWORD dwError= ERROR_SUCCESS;
+  int argStart = 1;
+
+  static const SERVICE_TABLE_ENTRY serviceTable[] = {
+    { SVCNAME, (LPSERVICE_MAIN_FUNCTION) SvcMain },
+    { NULL, NULL }
+    };
+
+  ghEventLog = RegisterEventSource(NULL, SVCNAME);
+  if (NULL == ghEventLog) {
+    dwError = GetLastError();
+    CHECK_SVC_STATUS_DONE(dwError, L"RegisterEventSource")
+  }
+
+  if (!StartServiceCtrlDispatcher(serviceTable)) {
+    dwError = GetLastError();
+    CHECK_SVC_STATUS_DONE(dwError, L"StartServiceCtrlDispatcher")
+  }
+
+done:
+  return dwError;
+}
+
+//----------------------------------------------------------------------------
+// Function: SvcMain
+//
+// Description:
+//  Service main entry point.
+//
+VOID WINAPI SvcMain() {
+  DWORD dwError = ERROR_SUCCESS;
+
+  gSvcStatusHandle = RegisterServiceCtrlHandler( 
+        SVCNAME, 
+        SvcCtrlHandler);
+  if( !gSvcStatusHandle ) { 
+    dwError = GetLastError();
+    CHECK_SVC_STATUS_DONE(dwError, L"RegisterServiceCtrlHandler")
+  } 
+  
+  // These SERVICE_STATUS members remain as set here
+  gSvcStatus.dwServiceType = SERVICE_WIN32_OWN_PROCESS; 
+  gSvcStatus.dwServiceSpecificExitCode = 0;    
+
+  // Report initial status to the SCM
+  ReportSvcStatus( SERVICE_START_PENDING, NO_ERROR, 3000 );
+
+  // Perform service-specific initialization and work.
+  dwError = SvcInit();
+  
+done:
+  return;
+}
+
+//----------------------------------------------------------------------------
+// Function: SvcInit
+//
+// Description:
+//  Initializes the service.
+//
+DWORD SvcInit() {
+  DWORD dwError = ERROR_SUCCESS;
+
+  dwError = EnableImpersonatePrivileges();
+  if( dwError != ERROR_SUCCESS ) {
+    ReportErrorCode(L"EnableImpersonatePrivileges", dwError);
+    goto done;
+  }
+
+  // The recommended way to shutdown the service is to use an event
+  //  and attach a callback with RegisterWaitForSingleObject
+  //
+  ghSvcStopEvent = CreateEvent(
+                           NULL,    // default security attributes
+                           TRUE,    // manual reset event
+                           FALSE,   // not signaled
+                           NULL);   // no name
+  
+  if ( ghSvcStopEvent == NULL)
+  {
+      dwError = GetLastError();
+      ReportSvcCheckError(EVENTLOG_ERROR_TYPE, SERVICE_CATEGORY, 
+        dwError, L"CreateEvent");
+      ReportSvcStatus( SERVICE_STOPPED, dwError, 0 );
+      goto done;
+  }
+
+  if (!RegisterWaitForSingleObject (&ghWaitObject,
+                            ghSvcStopEvent,
+                            SvcShutdown,
+                            NULL,
+                            INFINITE,
+                            WT_EXECUTEONLYONCE)) {
+    dwError = GetLastError();
+    ReportSvcCheckError(EVENTLOG_ERROR_TYPE, SERVICE_CATEGORY, 
+      dwError, L"RegisterWaitForSingleObject");
+    CloseHandle(ghSvcStopEvent);
+    ReportSvcStatus( SERVICE_STOPPED, dwError, 0 );
+    goto done;
+  }
+
+  dwError = ValidateConfigurationFile();
+  if (ERROR_SUCCESS != dwError) {
+    LogDebugMessage(L"ValidateConfigurationFile failed: %d", dwError);
+    SvcError(dwError);
+    goto done;
+  }
+
+  dwError = AuthInit();
+  if (ERROR_SUCCESS != dwError) {
+    LogDebugMessage(L"AuthInit failed: %d", dwError);
+    SvcError(dwError);
+    goto done;
+  }
+
+  dwError = InitLocalDirs();
+  if (ERROR_SUCCESS != dwError) {
+    LogDebugMessage(L"InitLocalDirs failed: %d", dwError);
+    SvcError(dwError);
+    goto done;
+  }
+
+  dwError = InitJobName();
+  if (ERROR_SUCCESS != dwError) {
+    LogDebugMessage(L"InitJobName failed: %d", dwError);
+    SvcError(dwError);
+    goto done;
+  }
+
+  // Report running status when initialization is  complete.
+  ReportSvcStatus( SERVICE_RUNNING, NO_ERROR, 0 );
+
+  dwError = RpcInit();
+
+done:
+  return dwError;
+}
+
+//----------------------------------------------------------------------------
+// Function: RpcAuthorizeCallback
+//
+// Description:
+//  RPC Authorization callback.
+//
+// Returns:
+//  RPC_S_OK for access authorized
+//  RPC_S_ACCESS_DENIED for access denied
+//
+RPC_STATUS CALLBACK RpcAuthorizeCallback (
+  RPC_IF_HANDLE  hInterface,
+  void* pContext) 
+{
+  RPC_STATUS                status, 
+                            unwindStatus, 
+                            authStatus = RPC_S_ACCESS_DENIED;
+  DWORD                     dwError;
+  LUID                      luidReserved2;
+  AUTHZ_ACCESS_REQUEST      request;
+  AUTHZ_ACCESS_REPLY        reply;
+  AUTHZ_CLIENT_CONTEXT_HANDLE hClientContext = NULL;
+  DWORD                     authError = ERROR_SUCCESS;
+  DWORD                     saclResult = 0;
+  ACCESS_MASK               grantedMask = 0;
+
+  ZeroMemory(&luidReserved2, sizeof(luidReserved2));
+  ZeroMemory(&request, sizeof(request));
+  ZeroMemory(&reply, sizeof(reply));
+  
+  status = RpcGetAuthorizationContextForClient(NULL,
+        FALSE,         // ImpersonateOnReturn
+        NULL,          // Reserved1
+        NULL,          // pExpirationTime
+        luidReserved2, // Reserved2
+        0,             // Reserved3
+        NULL,          // Reserved4
+        &hClientContext);
+  CHECK_RPC_STATUS_DONE(status, L"RpcGetAuthorizationContextForClient");
+
+  request.DesiredAccess = MAXIMUM_ALLOWED;  
+  reply.Error = &authError;
+  reply.SaclEvaluationResults = &saclResult;
+  reply.ResultListLength = 1;
+  reply.GrantedAccessMask = &grantedMask;
+
+  if (!AuthzAccessCheck(
+    0,
+    hClientContext,
+    &request,
+    NULL,   // AuditEvent
+    pAllowedSD,
+    NULL,  // OptionalSecurityDescriptorArray
+    0,     // OptionalSecurityDescriptorCount
+    &reply,
+    NULL  // phAccessCheckResults 
+    )) {
+    dwError = GetLastError();
+    CHECK_SVC_STATUS_DONE(dwError, L"AuthzAccessCheck");
+  }
+
+  LogDebugMessage(L"AutzAccessCheck: Error:%d sacl:%d access:%d\n", 
+    authError, saclResult, grantedMask);
+  if (authError == ERROR_SUCCESS && (grantedMask & SERVICE_ACCESS_MASK)) {
+    authStatus = RPC_S_OK;
+  }
+  
+done:
+  if (NULL != hClientContext) CHECK_UNWIND_RPC(RpcFreeAuthorizationContext(&hClientContext));
+  return authStatus;
+}
+
+//----------------------------------------------------------------------------
+// Function: AuthInit
+//
+// Description:
+//  Initializes the authorization structures (security descriptor).
+//
+// Notes:
+//  This is called from RunService solely for debugging purposed 
+//   so that it can be tested by wimply running winutil service from CLI (no SCM)
+//
+DWORD AuthInit() {
+  DWORD       dwError = ERROR_SUCCESS;
+  int         count = 0;
+  int         crt  = 0;
+  size_t      len = 0;
+  LPCWSTR     value = NULL;
+  WCHAR**     tokens = NULL;
+  LPWSTR      lpszSD = NULL;
+  ULONG       cchSD = 0;
+  DWORD       dwBufferSize = 0;
+  int         allowedCount = 0;
+  PSID*       allowedSids = NULL;
+  
+
+  dwError = GetConfigValue(
+    wsceConfigRelativePath,
+    NM_WSCE_ALLOWED, &len, &value);
+  CHECK_SVC_STATUS_DONE(dwError, L"GetConfigValue");
+
+  if (0 == len) {
+    dwError = ERROR_BAD_CONFIGURATION;
+    CHECK_SVC_STATUS_DONE(dwError, NM_WSCE_ALLOWED);
+  }
+  
+  dwError = SplitStringIgnoreSpaceW(len, value, L',', &count, &tokens);
+  CHECK_SVC_STATUS_DONE(dwError, L"SplitStringIgnoreSpaceW");
+
+  allowedSids = (PSID*) LocalAlloc(LPTR, sizeof(PSID) * count);
+  if (NULL == allowedSids) {
+    dwError = ERROR_OUTOFMEMORY;
+    CHECK_SVC_STATUS_DONE(dwError, L"LocalAlloc");
+  }
+  
+  for (crt = 0; crt < count; ++crt) {
+    dwError = GetSidFromAcctNameW(tokens[crt], &allowedSids[crt]);
+    CHECK_SVC_STATUS_DONE(dwError, L"GetSidFromAcctNameW");
+  }
+
+  allowedCount = count;
+  
+  dwError = BuildServiceSecurityDescriptor(SERVICE_ACCESS_MASK,
+    allowedCount, allowedSids, 0, NULL, NULL, &pAllowedSD);
+  CHECK_SVC_STATUS_DONE(dwError, L"BuildServiceSecurityDescriptor");
+  
+done:
+  if (lpszSD) LocalFree(lpszSD);
+  if (value) LocalFree(value);
+  if (tokens) LocalFree(tokens);
+  return dwError;
+}
+
+//----------------------------------------------------------------------------
+// Function: RpcInit
+//
+// Description:
+//  Initializes the RPC infrastructure and starts the RPC listenner.
+//
+DWORD RpcInit() {
+  RPC_STATUS  status;
+  DWORD       dwError;
+
+  status = RpcServerUseProtseqIf(SVCBINDING, 
+                 RPC_C_LISTEN_MAX_CALLS_DEFAULT,
+                 HadoopWinutilSvc_v1_0_s_ifspec,
+                 NULL);
+  if (RPC_S_OK != status) {
+    ReportSvcCheckError(EVENTLOG_ERROR_TYPE, SERVICE_CATEGORY, 
+      status, L"RpcServerUseProtseqIf");
+    SvcError(status);
+    dwError = status;
+    goto done;
+  }
+
+  status = RpcServerRegisterIfEx(HadoopWinutilSvc_v1_0_s_ifspec,
+                 NULL,                                          // MgrTypeUuid
+                 NULL,                                          // MgrEpv
+                 RPC_IF_ALLOW_LOCAL_ONLY,                       // Flags
+                 RPC_C_LISTEN_MAX_CALLS_DEFAULT,                // Max calls
+                 RpcAuthorizeCallback);                         // Auth callback
+  
+  if (RPC_S_OK != status) {
+    ReportSvcCheckError(EVENTLOG_ERROR_TYPE, SERVICE_CATEGORY, 
+      status, L"RpcServerRegisterIfEx");
+    SvcError(status);
+    dwError = status;
+    goto done;
+  }
+
+  status = RpcServerListen(1, RPC_C_LISTEN_MAX_CALLS_DEFAULT, TRUE);
+  if (RPC_S_ALREADY_LISTENING == status) {
+    ReportSvcCheckError(EVENTLOG_WARNING_TYPE, SERVICE_CATEGORY, 
+      status, L"RpcServerListen");
+  }
+  else if (RPC_S_OK != status) {
+    ReportSvcCheckError(EVENTLOG_ERROR_TYPE, SERVICE_CATEGORY, 
+      status, L"RpcServerListen");
+    SvcError(status);
+    dwError = status;
+    goto done;
+  }
+
+  isListenning = TRUE;
+  
+  ReportSvcMessage(EVENTLOG_INFORMATION_TYPE, SERVICE_CATEGORY, 
+      MSG_RPC_SERVICE_HAS_STARTED);
+  
+done:
+  return dwError;
+}
+
+//----------------------------------------------------------------------------
+// Function: RpcStop
+//
+// Description:
+//  Tears down the RPC infrastructure and stops the RPC listenner.
+//
+VOID RpcStop() {
+  RPC_STATUS  status;
+  
+  if (isListenning) {
+
+    status = RpcMgmtStopServerListening(NULL);
+    isListenning = FALSE;
+    
+    if (RPC_S_OK != status) {
+      ReportSvcCheckError(EVENTLOG_WARNING_TYPE, SERVICE_CATEGORY, 
+        status, L"RpcMgmtStopServerListening");
+    }
+  
+    ReportSvcMessage(EVENTLOG_INFORMATION_TYPE, SERVICE_CATEGORY, 
+        MSG_RPC_SERVICE_HAS_STOPPED);
+  }
+}
+
+//----------------------------------------------------------------------------
+// Function: CleanupHandles
+//
+// Description:
+//  Cleans up the global service handles.
+//
+VOID CleanupHandles() {
+  if (INVALID_HANDLE_VALUE != ghWaitObject) {
+    UnregisterWait(ghWaitObject);
+    ghWaitObject = INVALID_HANDLE_VALUE;
+  }
+  if (INVALID_HANDLE_VALUE != ghSvcStopEvent) {
+    CloseHandle(ghSvcStopEvent);
+    ghSvcStopEvent = INVALID_HANDLE_VALUE;
+  }
+  if (INVALID_HANDLE_VALUE != ghEventLog) {
+    DeregisterEventSource(ghEventLog);
+    ghEventLog = INVALID_HANDLE_VALUE;
+  }
+}
+
+//----------------------------------------------------------------------------
+// Function: SvcError
+//
+// Description:
+//  Aborts the startup sequence. Reports error, stops RPC, cleans up globals.
+//
+VOID SvcError(DWORD dwError) {
+  RpcStop();
+  CleanupHandles();
+  ReportSvcStatus( SERVICE_STOPPED, dwError, 0 );
+}
+
+//----------------------------------------------------------------------------
+// Function: SvcShutdown
+//
+// Description:
+//  Callback when the shutdown event is signaled. Stops RPC, cleans up globals.
+//
+VOID CALLBACK SvcShutdown(
+  _In_  PVOID lpParameter,
+  _In_  BOOLEAN TimerOrWaitFired) {
+  RpcStop();
+  CleanupHandles();
+  ReportSvcStatus( SERVICE_STOPPED, NO_ERROR, 0 );
+}
+
+//----------------------------------------------------------------------------
+// Function: SvcCtrlHandler
+//
+// Description:
+//  Callback from SCM for for service events (signals).
+//
+// Notes:
+//   Shutdown is indirect, we set her the STOP_PENDING state and signal the stop event.
+//   Signaling the event invokes SvcShutdown which completes the shutdown.
+//   This two staged approach allows the SCM handler to complete fast, 
+//   not blocking the SCM big fat global lock.
+//
+VOID WINAPI SvcCtrlHandler( DWORD dwCtrl )
+{
+   // Handle the requested control code. 
+
+   switch(dwCtrl) 
+   {  
+      case SERVICE_CONTROL_STOP: 
+         ReportSvcStatus(SERVICE_STOP_PENDING, NO_ERROR, 0);
+
+         // Signal the service to stop.
+         SetEvent(ghSvcStopEvent);
+         
+         return;
+ 
+      default: 
+         break;
+   } 
+   
+}
+
+//----------------------------------------------------------------------------
+// Function: ReportSvcStatus
+//
+// Description:
+//  Updates the service status with the SCM.
+//
+VOID ReportSvcStatus( DWORD dwCurrentState,
+                      DWORD dwWin32ExitCode,
+                      DWORD dwWaitHint)
+{
+    static DWORD dwCheckPoint = 1;
+    DWORD dwError;
+
+    // Fill in the SERVICE_STATUS structure.
+
+    gSvcStatus.dwCurrentState = dwCurrentState;
+    gSvcStatus.dwWin32ExitCode = dwWin32ExitCode;
+    gSvcStatus.dwWaitHint = dwWaitHint;
+
+    if (dwCurrentState == SERVICE_START_PENDING)
+        gSvcStatus.dwControlsAccepted = 0;
+    else gSvcStatus.dwControlsAccepted = SERVICE_ACCEPT_STOP;
+
+    if ( (dwCurrentState == SERVICE_RUNNING) ||
+           (dwCurrentState == SERVICE_STOPPED) )
+        gSvcStatus.dwCheckPoint = 0;
+    else gSvcStatus.dwCheckPoint = dwCheckPoint++;
+
+    // Report the status of the service to the SCM.
+    if (!SetServiceStatus( gSvcStatusHandle, &gSvcStatus)) {
+      dwError = GetLastError();
+      ReportSvcCheckError(EVENTLOG_WARNING_TYPE, SERVICE_CATEGORY, 
+        dwError, L"SetServiceStatus");
+    };
+}
+
+//----------------------------------------------------------------------------
+// Function: WinutilsCreateProcessAsUser
+//
+// Description:
+//  The RPC midl declared function implementation
+//
+// Returns:
+// ERROR_SUCCESS: On success
+// Error code otherwise: otherwise
+//
+// Notes:
+//  This is the entry point when the NodeManager does the RPC call
+//  Note that the RPC call does not do any S4U work. Is simply spawns (suspended) wintutils
+//  using the right command line and the handles over the spwaned process to the NM
+//  The actual S4U work occurs in the spawned process, run and monitored by the NM
+//
+error_status_t WinutilsCreateProcessAsUser( 
+    /* [in] */ handle_t IDL_handle,
+    /* [in] */ int nmPid,
+    /* [in] */ CREATE_PROCESS_REQUEST *request,
+    /* [out] */ CREATE_PROCESS_RESPONSE **response) {
+
+  DWORD dwError = ERROR_SUCCESS;
+  LPCWSTR inserts[] = {request->cwd, request->jobName, request->user, request->pidFile, request->cmdLine, NULL};
+  WCHAR winutilsPath[MAX_PATH];
+  WCHAR fullCmdLine[32768];
+  HANDLE taskStdInRd = INVALID_HANDLE_VALUE, taskStdInWr = INVALID_HANDLE_VALUE,
+    taskStdOutRd = INVALID_HANDLE_VALUE, taskStdOutWr = INVALID_HANDLE_VALUE,
+    taskStdErrRd = INVALID_HANDLE_VALUE, taskStdErrWr = INVALID_HANDLE_VALUE,
+    hNmProcess = INVALID_HANDLE_VALUE,
+    hDuplicateProcess = INVALID_HANDLE_VALUE,
+    hDuplicateThread = INVALID_HANDLE_VALUE,
+    hDuplicateStdIn  = INVALID_HANDLE_VALUE,
+    hDuplicateStdOut = INVALID_HANDLE_VALUE,
+    hDuplicateStdErr = INVALID_HANDLE_VALUE,
+    hSelfProcess = INVALID_HANDLE_VALUE,
+    hJob = INVALID_HANDLE_VALUE;
+  BOOL fMustCleanupProcess = FALSE;
+  
+  HRESULT hr;
+  STARTUPINFO si;
+  PROCESS_INFORMATION pi;
+  SECURITY_ATTRIBUTES saTaskStdInOutErr;
+
+  ZeroMemory( &si, sizeof(si) );
+  si.cb = sizeof(si);
+  ZeroMemory( &pi, sizeof(pi) );
+  pi.hProcess = INVALID_HANDLE_VALUE;
+  pi.hThread = INVALID_HANDLE_VALUE;
+  ZeroMemory( &saTaskStdInOutErr, sizeof(saTaskStdInOutErr));
+  
+
+  if (gJobName) {
+    hJob = OpenJobObject(JOB_OBJECT_ASSIGN_PROCESS, FALSE, gJobName);
+    if (!hJob) {
+      dwError = GetLastError();
+      ReportSvcCheckError(EVENTLOG_ERROR_TYPE, SERVICE_CATEGORY, 
+        dwError, L"OpenJobObject");
+      goto done;
+    }
+  }
+
+
+  // NB: GetCurrentProcess returns a pseudo-handle that just so happens 
+  // has the value -1, ie. INVALID_HANDLE_VALUE. It cannot fail.
+  // 
+  hSelfProcess = GetCurrentProcess();
+
+  hNmProcess = OpenProcess(PROCESS_DUP_HANDLE, FALSE, nmPid);
+  if (NULL == hNmProcess) {
+    dwError = GetLastError();
+    ReportSvcCheckError(EVENTLOG_ERROR_TYPE, SERVICE_CATEGORY, 
+      dwError, L"OpenProcess");
+    goto done;
+  }
+
+  GetModuleFileName(NULL, winutilsPath, sizeof(winutilsPath)/sizeof(WCHAR));
+  dwError = GetLastError(); // Always check after GetModuleFileName for ERROR_INSSUFICIENT_BUFFER
+  if (dwError) {
+    ReportSvcCheckError(EVENTLOG_ERROR_TYPE, SERVICE_CATEGORY, 
+      dwError, L"GetModuleFileName");
+    goto done;
+  }
+
+  // NB. We can call CreateProcess("wintuls","task create ...") or we can call
+  // CreateProcess(NULL, "winutils task create"). Only the second form passes "task" as
+  // argv[1], as expected by main. First form passes "task" as argv[0] and main fails.
+  
+  hr = StringCbPrintf(fullCmdLine, sizeof(fullCmdLine), L"\"%s\" task createAsUser %ls %ls %ls %ls",
+    winutilsPath,
+    request->jobName, request->user, request->pidFile, request->cmdLine);
+  if (FAILED(hr)) {
+    ReportSvcCheckError(EVENTLOG_ERROR_TYPE, SERVICE_CATEGORY, 
+      hr, L"StringCbPrintf:fullCmdLine");
+    goto done;
+  }
+
+  LogDebugMessage(L"[%ls]: %ls %ls\n", request->cwd, winutilsPath, fullCmdLine);
+
+  // stdin/stdout/stderr redirection is handled here
+  // We create 3 anonymous named pipes. 
+  // Security attributes are required so that the handles can be inherited.
+  // We assign one end of the pipe to the process (stdin gets a read end, stdout gets a write end)
+  // We then duplicate the other end in the NM process, and we close our own handle
+  // Finally we return the duplicate handle values to the NM
+  // The NM will attach Java file dscriptors to the duplicated handles and 
+  // read/write them as ordinary Java InputStream/OutputStream objects
+
+  si.dwFlags |= STARTF_USESTDHANDLES;
+
+  saTaskStdInOutErr.nLength = sizeof(SECURITY_ATTRIBUTES); 
+  saTaskStdInOutErr.bInheritHandle = TRUE; 
+  saTaskStdInOutErr.lpSecurityDescriptor = NULL; 
+
+  if (!CreatePipe(&taskStdInRd, &taskStdInWr, &saTaskStdInOutErr, 0)) {
+    dwError = GetLastError();
+    goto done;
+  }
+  if (!SetHandleInformation(taskStdInWr, HANDLE_FLAG_INHERIT, FALSE)) {
+    dwError = GetLastError();
+    goto done;
+  }
+  si.hStdInput  = taskStdInRd;
+
+  if (!CreatePipe(&taskStdOutRd, &taskStdOutWr, &saTaskStdInOutErr, 0)) {
+    dwError = GetLastError();
+    goto done;
+  }
+  if (!SetHandleInformation(taskStdOutRd, HANDLE_FLAG_INHERIT, FALSE)) {
+    dwError = GetLastError();
+    goto done;
+  }
+  si.hStdOutput  = taskStdOutWr;
+
+  if (!CreatePipe(&taskStdErrRd, &taskStdErrWr, &saTaskStdInOutErr, 0)) {
+    dwError = GetLastError();
+    goto done;
+  }
+  if (!SetHandleInformation(taskStdErrRd, HANDLE_FLAG_INHERIT, FALSE)) {
+    dwError = GetLastError();
+    goto done;
+  }
+  si.hStdError  = taskStdErrWr;
+
+  if (!CreateProcess(
+    NULL,                     // lpApplicationName,
+    fullCmdLine,              // lpCommandLine,
+    NULL,                     // lpProcessAttributes,
+    NULL,                     // lpThreadAttributes,
+    TRUE,                     // bInheritHandles,
+    CREATE_SUSPENDED,         // dwCreationFlags,
+    NULL,                     // lpEnvironment,
+    request->cwd,             // lpCurrentDirectory,
+    &si,                      // lpStartupInfo
+    &pi)) {                   // lpProcessInformation
+    
+    dwError = GetLastError();
+    ReportSvcCheckError(EVENTLOG_ERROR_TYPE, SERVICE_CATEGORY, 
+      dwError, L"CreateProcess");
+    goto done;
+  }
+
+  fMustCleanupProcess = TRUE;
+
+  LogDebugMessage(L"CreateProcess: pid:%x\n", pi.dwProcessId);
+
+  if (INVALID_HANDLE_VALUE != hJob) {
+    if (!AssignProcessToJobObject(hJob, pi.hProcess)) {
+      dwError = GetLastError();
+      goto done;
+    }
+  }
+
+  // Grant full access to the container user on the 'winutils task createAsUser ...' helper process
+  dwError = AddNodeManagerAndUserACEsToObject(pi.hProcess, request->user, PROCESS_ALL_ACCESS);
+  if (dwError) {
+    LogDebugMessage(L"failed: AddNodeManagerAndUserACEsToObject\n");
+    goto done;
+  }
+
+  if (!DuplicateHandle(hSelfProcess, pi.hProcess, hNmProcess,
+    &hDuplicateProcess, 0, FALSE, DUPLICATE_SAME_ACCESS)) {
+    dwError = GetLastError();
+    LogDebugMessage(L"failed: pi.hProcess\n");
+    goto done;
+  }
+  
+  if (!DuplicateHandle(hSelfProcess, pi.hThread, hNmProcess,
+    &hDuplicateThread, 0, FALSE, DUPLICATE_SAME_ACCESS)) {
+    dwError = GetLastError();
+    LogDebugMessage(L"failed: pi.hThread\n");
+    goto done;
+  }
+
+  if (!DuplicateHandle(hSelfProcess, taskStdInWr, hNmProcess,
+    &hDuplicateStdIn, 0, FALSE, DUPLICATE_SAME_ACCESS)) {
+    dwError = GetLastError();
+    LogDebugMessage(L"failed: taskStdInWr\n");
+    goto done;
+  }
+
+  if (!DuplicateHandle(hSelfProcess, taskStdOutRd, hNmProcess,
+    &hDuplicateStdOut, 0, FALSE, DUPLICATE_SAME_ACCESS)) {
+    dwError = GetLastError();
+    LogDebugMessage(L"failed: taskStdOutRd\n");
+    goto done;
+  }
+
+  if (!DuplicateHandle(hSelfProcess, taskStdErrRd, hNmProcess,
+    &hDuplicateStdErr, 0, FALSE, DUPLICATE_SAME_ACCESS)) {
+    dwError = GetLastError();
+    LogDebugMessage(L"failed: taskStdErrRd\n");
+    goto done;
+  }
+
+  *response = (CREATE_PROCESS_RESPONSE*) MIDL_user_allocate(sizeof(CREATE_PROCESS_RESPONSE));
+  if (NULL == *response) {
+    dwError = ERROR_OUTOFMEMORY;
+    LogDebugMessage(L"Failed to allocate CREATE_PROCESS_RESPONSE* response\n");
+    goto done;
+  }
+
+  // We're now transfering ownership of the duplicated handles to the caller
+  // If the RPC call fails *after* this point the handles are leaked inside the NM process
+  // Note that there are no more API calls, only assignments. A failure could occur only if
+  // foced (process kill) or hardware error (faulty memory, processort bit flip etc).
+
+  (*response)->hProcess = hDuplicateProcess;
+  (*response)->hThread = hDuplicateThread;
+  (*response)->hStdIn = hDuplicateStdIn;
+  (*response)->hStdOut = hDuplicateStdOut;
+  (*response)->hStdErr = hDuplicateStdErr;
+
+  fMustCleanupProcess = FALSE;
+  
+done:
+
+  if (fMustCleanupProcess) {
+    LogDebugMessage(L"Cleaning process: %d due to error:%d\n", pi.dwProcessId, dwError);
+    TerminateProcess(pi.hProcess, EXIT_FAILURE);
+
+    // cleanup the duplicate handles inside the NM.
+
+    if (INVALID_HANDLE_VALUE != hDuplicateProcess) {
+      DuplicateHandle(hNmProcess, hDuplicateProcess, NULL, NULL, 0, FALSE, DUPLICATE_CLOSE_SOURCE);
+    }
+    if (INVALID_HANDLE_VALUE != hDuplicateThread) {
+      DuplicateHandle(hNmProcess, hDuplicateThread, NULL, NULL, 0, FALSE, DUPLICATE_CLOSE_SOURCE);
+    }
+    if (INVALID_HANDLE_VALUE != hDuplicateStdIn) {
+      DuplicateHandle(hNmProcess, hDuplicateStdIn, NULL, NULL, 0, FALSE, DUPLICATE_CLOSE_SOURCE);
+    }
+    if (INVALID_HANDLE_VALUE != hDuplicateStdOut) {
+      DuplicateHandle(hNmProcess, hDuplicateStdOut, NULL, NULL, 0, FALSE, DUPLICATE_CLOSE_SOURCE);
+    }
+    if (INVALID_HANDLE_VALUE != hDuplicateStdErr) {
+      DuplicateHandle(hNmProcess, hDuplicateStdErr, NULL, NULL, 0, FALSE, DUPLICATE_CLOSE_SOURCE);
+    }
+  }
+
+  if (INVALID_HANDLE_VALUE != hSelfProcess) CloseHandle(hSelfProcess);
+  if (INVALID_HANDLE_VALUE != hNmProcess) CloseHandle(hNmProcess);
+  if (INVALID_HANDLE_VALUE != taskStdInRd) CloseHandle(taskStdInRd);
+  if (INVALID_HANDLE_VALUE != taskStdInWr) CloseHandle(taskStdInWr);
+  if (INVALID_HANDLE_VALUE != taskStdOutRd) CloseHandle(taskStdOutRd);
+  if (INVALID_HANDLE_VALUE != taskStdOutWr) CloseHandle(taskStdOutWr);
+  if (INVALID_HANDLE_VALUE != taskStdErrRd) CloseHandle(taskStdErrRd);
+  if (INVALID_HANDLE_VALUE != taskStdErrWr) CloseHandle(taskStdErrWr);
+
+
+  // This is closing our own process/thread handles. 
+  // If the transfer was succesfull the NM has its own duplicates (if any)
+  if (INVALID_HANDLE_VALUE != pi.hThread) CloseHandle(pi.hThread);
+  if (INVALID_HANDLE_VALUE != pi.hProcess) CloseHandle(pi.hProcess);
+
+  if (hJob) CloseHandle(hJob);
+
+  return dwError;
+}
+
+error_status_t WinutilsCreateFile(
+  /* [in] */ handle_t IDL_handle,
+  /* [in] */ int nm_pid,
+  /* [in] */ CREATEFILE_REQUEST *request,
+  /* [out] */ CREATEFILE_RESPONSE **response) {
+
+  DWORD dwError = ERROR_SUCCESS;
+
+  HANDLE hNmProcess = INVALID_HANDLE_VALUE, 
+    hFile = INVALID_HANDLE_VALUE,
+    hDuplicateFile = INVALID_HANDLE_VALUE,
+    hSelfProcess = GetCurrentProcess();
+
+  SECURITY_ATTRIBUTES saFile;
+
+  ZeroMemory( &saFile, sizeof(saFile)); 
+
+  dwError = ValidateLocalPath(request->path);
+  CHECK_SVC_STATUS_DONE(dwError,L"ValidateLocalPath request->path");    
+
+  saFile.nLength = sizeof(SECURITY_ATTRIBUTES); 
+  saFile.bInheritHandle = TRUE; 
+  saFile.lpSecurityDescriptor = NULL;
+
+  hFile = CreateFile(
+    request->path,
+    request->desiredAccess,
+    request->shareMode,
+    &saFile,
+    request->creationDisposition,
+    request->flags,
+    NULL); // hTemplate
+  if (INVALID_HANDLE_VALUE == hFile) {
+    dwError = GetLastError();
+    goto done;
+  }
+
+  hNmProcess = OpenProcess(PROCESS_DUP_HANDLE, FALSE, nm_pid);
+  if (NULL == hNmProcess) {
+    dwError = GetLastError();
+    goto done;
+  }
+
+  if (!DuplicateHandle(hSelfProcess, hFile,
+    hNmProcess, &hDuplicateFile,
+    0, FALSE, DUPLICATE_SAME_ACCESS)) {
+    dwError = GetLastError();
+    goto done;
+  }
+
+  *response = (CREATEFILE_RESPONSE*) MIDL_user_allocate(sizeof(CREATEFILE_RESPONSE));
+  if (NULL == *response) {
+    dwError = ERROR_OUTOFMEMORY;
+    goto done;
+  }
+
+  (*response)->hFile = hDuplicateFile;
+  hDuplicateFile = INVALID_HANDLE_VALUE;
+
+done:
+
+  if (INVALID_HANDLE_VALUE != hFile) CloseHandle(hFile);
+  if (INVALID_HANDLE_VALUE != hDuplicateFile) {
+    DuplicateHandle(hNmProcess, hDuplicateFile, NULL, NULL, 0, FALSE, DUPLICATE_CLOSE_SOURCE);
+  }
+  if (INVALID_HANDLE_VALUE != hNmProcess) CloseHandle(hNmProcess);
+
+  LogDebugMessage(L"WinutilsCreateFile: %s %d, %d, %d, %d: %d",
+    request->path,
+    request->desiredAccess,
+    request->shareMode,
+    request->creationDisposition,
+    request->flags,
+    dwError);
+  
+  return dwError;
+}
+
+error_status_t WinutilsKillTask( 
+    /* [in] */ handle_t IDL_handle,
+    /* [in] */ KILLTASK_REQUEST *request) {
+  DWORD dwError = ERROR_SUCCESS;
+  HRESULT hr;
+  WCHAR bufferName[MAX_PATH];
+
+  dwError = GetSecureJobObjectName(request->taskName, MAX_PATH, bufferName);
+  CHECK_SVC_STATUS_DONE(dwError, L"GetSecureJobObjectName");
+
+  dwError = KillTask(bufferName);
+
+  if (ERROR_ACCESS_DENIED == dwError) {
+    // This process runs as LocalSystem with debug privilege enabled
+    // The job has a security descriptor that explictly grants JOB_OBJECT_ALL_ACCESS to us
+    // If we get ACCESS DENIED it means the job is being unwound
+    dwError = ERROR_SUCCESS;
+  }
+  
+done:  
+  LogDebugMessage(L"WinutilsKillTask: %s :%d\n", bufferName, dwError);
+  return dwError;
+}
+
+
+error_status_t WinutilsDeletePath(
+  /* [in] */ handle_t IDL_handle,
+  /* [in] */ DELETEPATH_REQUEST *request,
+  /* [out] */ DELETEPATH_RESPONSE **response) {
+
+  DWORD dwError = ERROR_SUCCESS;
+  BOOL deleted = FALSE;
+
+  dwError = ValidateLocalPath(request->path);
+  CHECK_SVC_STATUS_DONE(dwError,L"ValidateLocalPath request->path");
+
+  switch(request->type) {
+  case PATH_IS_DIR:
+    deleted = RemoveDirectory(request->path);
+    if (!deleted) {
+      LogDebugMessage(L"Error %d deleting directory %s\n", GetLastError(), request->path);
+    }
+    break;
+  case PATH_IS_FILE:
+    deleted = DeleteFile(request->path);
+    if (!deleted) {
+      LogDebugMessage(L"Error %d deleting file %s\n", GetLastError(), request->path);
+    }
+    break;
+  default:
+    dwError = ERROR_BAD_ARGUMENTS;
+    CHECK_SVC_STATUS_DONE(dwError, L"request->operation");
+  }
+
+  *response = (DELETEPATH_RESPONSE*) MIDL_user_allocate(sizeof(DELETEPATH_RESPONSE));
+  if (NULL == *response) {
+    dwError = ERROR_OUTOFMEMORY;
+    CHECK_SVC_STATUS_DONE(dwError, L"MIDL_user_allocate");
+  }
+
+  (*response)->deleted = deleted;
+
+done:
+
+  LogDebugMessage(L"WinutilsDeletePath: %s %d: %d %d",
+    request->path,
+    request->type,
+    deleted,
+    dwError);
+  
+  return dwError;
+}
+
+error_status_t WinutilsMkDir( 
+    /* [in] */ handle_t IDL_handle,
+    /* [in] */ MKDIR_REQUEST *request) {
+  DWORD dwError = ERROR_SUCCESS;
+
+  dwError = ValidateLocalPath(request->filePath);
+  CHECK_SVC_STATUS_DONE(dwError,L"ValidateLocalPath request->filePath");  
+
+  if (!CreateDirectory(request->filePath, NULL)) {
+    dwError = GetLastError();
+    CHECK_SVC_STATUS_DONE(dwError, L"CreateDirectory");
+  }
+  
+done:  
+  LogDebugMessage(L"WinutilsMkDir: %s :%d\n", request->filePath, dwError);
+  return dwError;
+}
+
+error_status_t WinutilsChown( 
+    /* [in] */ handle_t IDL_handle,
+    /* [in] */ CHOWN_REQUEST *request) {
+  DWORD dwError = ERROR_SUCCESS;
+
+  dwError = ValidateLocalPath(request->filePath);
+  CHECK_SVC_STATUS_DONE(dwError,L"ValidateLocalPath request->filePath");
+  
+  dwError = ChownImpl(request->ownerName, request->groupName, request->filePath);
+  CHECK_SVC_STATUS_DONE(dwError, L"ChownImpl");
+
+done:  
+  LogDebugMessage(L"WinutilsChown: %s %s %s :%d\n",
+    request->ownerName, request->groupName, request->filePath, dwError);
+  return dwError;
+}
+
+error_status_t WinutilsChmod( 
+    /* [in] */ handle_t IDL_handle,
+    /* [in] */ CHMOD_REQUEST *request) {
+  DWORD dwError = ERROR_SUCCESS;
+
+  dwError = ValidateLocalPath(request->filePath);
+  CHECK_SVC_STATUS_DONE(dwError,L"ValidateLocalPath request->filePath");
+  
+  dwError = ChangeFileModeByMask(request->filePath, request->mode);
+  CHECK_SVC_STATUS_DONE(dwError, L"ChangeFileModeByMask");
+
+done:
+  LogDebugMessage(L"WinutilsChmod: %s %o :%d\n",
+   request->filePath, request->mode, dwError);
+  return dwError;
+}
+
+error_status_t WinutilsMoveFile( 
+    /* [in] */ handle_t IDL_handle,
+    /* [in] */ MOVEFILE_REQUEST *request) {
+  DWORD dwError = ERROR_SUCCESS;
+  DWORD flags = 0;
+
+  dwError = ValidateLocalPath(request->sourcePath);
+  CHECK_SVC_STATUS_DONE(dwError,L"ValidateLocalPath request->sourcePath");
+
+  dwError = ValidateLocalPath(request->destinationPath);
+  CHECK_SVC_STATUS_DONE(dwError,L"ValidateLocalPath request->destinationPath");
+
+  switch (request->operation) {
+  case MOVE_FILE:
+    flags |= MOVEFILE_COPY_ALLOWED;
+    if (request->replaceExisting) flags |= MOVEFILE_REPLACE_EXISTING;
+    if (!MoveFileEx(request->sourcePath, request->destinationPath, flags)) {
+      dwError = GetLastError();
+      CHECK_SVC_STATUS_DONE(dwError, L"MoveFileEx");
+    }
+    break;
+  case COPY_FILE:
+    if (!request->replaceExisting) flags |= COPY_FILE_FAIL_IF_EXISTS;
+    if (!CopyFileEx(request->sourcePath, request->destinationPath,
+          NULL, // lpProgressRoutine
+          NULL, // lpData
+          NULL, // pbCancel
+          flags)) {
+      dwError = GetLastError();
+      CHECK_SVC_STATUS_DONE(dwError, L"CopyFileEx");
+    }
+    break;
+  default:
+    dwError = ERROR_BAD_ARGUMENTS;
+    CHECK_SVC_STATUS_DONE(dwError, L"request->operation");
+  }
+
+done:  
+  LogDebugMessage(L"WinutilsMoveFile: %d: %s %s :%d\n",
+    request->operation, request->sourcePath, request->destinationPath, dwError);
+  return dwError;
+}
+
+
+//----------------------------------------------------------------------------
+// Function: ServiceUsage
+//
+// Description:
+//  Prints the CLI arguments for service command.
+//
+void ServiceUsage()
+{
+  fwprintf(stdout, L"\
+    Usage: service\n\
+    Starts the nodemanager Windows Secure Container Executor helper service.\n\
+    The service must run as a high privileged account (LocalSystem)\n\
+    and is used by the nodemanager WSCE to spawn secure containers on Windows.\n");
+}
+
+

파일 크기가 너무 크기때문에 변경 상태를 표시하지 않습니다.
+ 751 - 237
hadoop-common-project/hadoop-common/src/main/winutils/task.c


+ 64 - 0
hadoop-common-project/hadoop-common/src/main/winutils/winutils.mc

@@ -0,0 +1,64 @@
+;/*
+; * 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.
+; */
+
+; // winutils.mc 
+
+; // EventLog messages for Hadoop winutils service.
+
+
+LanguageNames=(English=0x409:MSG00409)
+
+
+; // The following are the categories of events.
+
+MessageIdTypedef=WORD
+
+MessageId=0x1
+SymbolicName=SERVICE_CATEGORY
+Language=English
+Service Events
+.
+
+MessageId=0x2
+SymbolicName=LOG_CATEGORY
+Language=English
+Task Events
+.
+
+; // The following are the message definitions.
+
+MessageIdTypedef=DWORD
+
+MessageId=0x80
+SymbolicName=MSG_CHECK_ERROR
+Language=English
+%1. Error %2: %3.
+.
+
+MessageId=0x100
+SymbolicName=MSG_RPC_SERVICE_HAS_STARTED
+Language=English
+The LPC server is listenning.
+.
+
+MessageId=0x200
+SymbolicName=MSG_RPC_SERVICE_HAS_STOPPED
+Language=English
+The LPC server has stopped listenning.
+.
+

+ 4 - 14
hadoop-common-project/hadoop-common/src/main/winutils/winutils.sln

@@ -26,26 +26,16 @@ Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "libwinutils", "libwinutils.
 EndProject
 Global
 	GlobalSection(SolutionConfigurationPlatforms) = preSolution
-		Debug|Win32 = Debug|Win32
 		Debug|x64 = Debug|x64
-		Release|Win32 = Release|Win32
 		Release|x64 = Release|x64
 	EndGlobalSection
 	GlobalSection(ProjectConfigurationPlatforms) = postSolution
-		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Debug|Win32.ActiveCfg = Debug|x64
-		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Debug|Win32.Build.0 = Debug|x64
-		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Debug|x64.ActiveCfg = Debug|x64
-		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Debug|x64.Build.0 = Debug|x64
-		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Release|Win32.ActiveCfg = Release|Win32
-		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Release|Win32.Build.0 = Release|Win32
+		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Debug|x64.ActiveCfg = Release|x64
+		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Debug|x64.Build.0 = Release|x64
 		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Release|x64.ActiveCfg = Release|x64
 		{D94B3BD7-39CC-47A0-AE9A-353FDE506F33}.Release|x64.Build.0 = Release|x64
-		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Debug|Win32.ActiveCfg = Debug|x64
-		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Debug|Win32.Build.0 = Debug|x64
-		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Debug|x64.ActiveCfg = Debug|x64
-		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Debug|x64.Build.0 = Debug|x64
-		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Release|Win32.ActiveCfg = Release|Win32
-		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Release|Win32.Build.0 = Release|Win32
+		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Debug|x64.ActiveCfg = Release|x64
+		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Debug|x64.Build.0 = Release|x64
 		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Release|x64.ActiveCfg = Release|x64
 		{12131AA7-902E-4A6D-9CE3-043261D22A12}.Release|x64.Build.0 = Release|x64
 	EndGlobalSection

+ 36 - 70
hadoop-common-project/hadoop-common/src/main/winutils/winutils.vcxproj

@@ -19,18 +19,10 @@
 
 <Project DefaultTargets="Build" ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
   <ItemGroup Label="ProjectConfigurations">
-    <ProjectConfiguration Include="Debug|Win32">
-      <Configuration>Debug</Configuration>
-      <Platform>Win32</Platform>
-    </ProjectConfiguration>
     <ProjectConfiguration Include="Debug|x64">
       <Configuration>Debug</Configuration>
       <Platform>x64</Platform>
     </ProjectConfiguration>
-    <ProjectConfiguration Include="Release|Win32">
-      <Configuration>Release</Configuration>
-      <Platform>Win32</Platform>
-    </ProjectConfiguration>
     <ProjectConfiguration Include="Release|x64">
       <Configuration>Release</Configuration>
       <Platform>x64</Platform>
@@ -42,22 +34,11 @@
     <RootNamespace>winutils</RootNamespace>
   </PropertyGroup>
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.Default.props" />
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'" Label="Configuration">
-    <ConfigurationType>Application</ConfigurationType>
-    <UseDebugLibraries>true</UseDebugLibraries>
-    <CharacterSet>Unicode</CharacterSet>
-  </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'" Label="Configuration">
     <ConfigurationType>Application</ConfigurationType>
     <UseDebugLibraries>true</UseDebugLibraries>
     <CharacterSet>Unicode</CharacterSet>
   </PropertyGroup>
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="Configuration">
-    <ConfigurationType>Application</ConfigurationType>
-    <UseDebugLibraries>false</UseDebugLibraries>
-    <WholeProgramOptimization>true</WholeProgramOptimization>
-    <CharacterSet>Unicode</CharacterSet>
-  </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="Configuration">
     <ConfigurationType>Application</ConfigurationType>
     <UseDebugLibraries>false</UseDebugLibraries>
@@ -67,15 +48,9 @@
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.props" />
   <ImportGroup Label="ExtensionSettings">
   </ImportGroup>
-  <ImportGroup Label="PropertySheets" Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
-    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
-  </ImportGroup>
   <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'" Label="PropertySheets">
     <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
   </ImportGroup>
-  <ImportGroup Label="PropertySheets" Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
-    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
-  </ImportGroup>
   <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="PropertySheets">
     <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
   </ImportGroup>
@@ -83,57 +58,32 @@
   <PropertyGroup>
     <IncludePath>include;$(IncludePath)</IncludePath>
   </PropertyGroup>
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
-    <LinkIncremental>true</LinkIncremental>
-  </PropertyGroup>
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'">
-    <LinkIncremental>true</LinkIncremental>
-    <OutDir />
-    <IntDir>..\..\..\target\winutils\$(Configuration)\</IntDir>
-  </PropertyGroup>
-  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
-    <LinkIncremental>false</LinkIncremental>
-  </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
     <LinkIncremental>false</LinkIncremental>
-    <IntDir>..\..\..\target\winutils\$(Platform)\$(Configuration)\</IntDir>
-    <OutDir>..\..\..\target\bin\</OutDir>
   </PropertyGroup>
-  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
-    <ClCompile>
-      <PrecompiledHeader>
-      </PrecompiledHeader>
-      <WarningLevel>Level3</WarningLevel>
-      <Optimization>Disabled</Optimization>
-      <PreprocessorDefinitions>WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions)</PreprocessorDefinitions>
-    </ClCompile>
-    <Link>
-      <SubSystem>Console</SubSystem>
-      <GenerateDebugInformation>true</GenerateDebugInformation>
-    </Link>
-  </ItemDefinitionGroup>
   <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'">
     <ClCompile>
       <PrecompiledHeader>
       </PrecompiledHeader>
       <WarningLevel>Level4</WarningLevel>
       <Optimization>Disabled</Optimization>
-      <PreprocessorDefinitions>WIN32;_DEBUG;_CONSOLE;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <PreprocessorDefinitions>WIN32;_CONSOLE;_DEBUG;_UNICODE;UNICODE;WSCE_CONFIG_DIR=$(WsceConfigDir);WSCE_CONFIG_FILE=$(WsceConfigFile);%(PreprocessorDefinitions)</PreprocessorDefinitions>
     </ClCompile>
     <Link>
       <SubSystem>Console</SubSystem>
       <GenerateDebugInformation>true</GenerateDebugInformation>
     </Link>
   </ItemDefinitionGroup>
-  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
     <ClCompile>
       <WarningLevel>Level3</WarningLevel>
       <PrecompiledHeader>
       </PrecompiledHeader>
-      <Optimization>MaxSpeed</Optimization>
+      <!-- <Optimization>MaxSpeed</Optimization> -->
+      <Optimization>Disabled</Optimization>
       <FunctionLevelLinking>true</FunctionLevelLinking>
       <IntrinsicFunctions>true</IntrinsicFunctions>
-      <PreprocessorDefinitions>WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <PreprocessorDefinitions>WIN32;_CONSOLE;NDEBUG;_UNICODE;UNICODE;WSCE_CONFIG_DIR=$(WsceConfigDir);WSCE_CONFIG_FILE=$(WsceConfigFile);%(PreprocessorDefinitions)</PreprocessorDefinitions>
     </ClCompile>
     <Link>
       <SubSystem>Console</SubSystem>
@@ -142,24 +92,40 @@
       <OptimizeReferences>true</OptimizeReferences>
     </Link>
   </ItemDefinitionGroup>
-  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
+  <ItemDefinitionGroup>
     <ClCompile>
-      <WarningLevel>Level3</WarningLevel>
-      <PrecompiledHeader>
-      </PrecompiledHeader>
-      <Optimization>MaxSpeed</Optimization>
-      <FunctionLevelLinking>true</FunctionLevelLinking>
-      <IntrinsicFunctions>true</IntrinsicFunctions>
-      <PreprocessorDefinitions>WIN32;NDEBUG;_CONSOLE;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <AdditionalIncludeDirectories>$(IntermediateOutputPath)</AdditionalIncludeDirectories>
     </ClCompile>
-    <Link>
-      <SubSystem>Console</SubSystem>
-      <GenerateDebugInformation>true</GenerateDebugInformation>
-      <EnableCOMDATFolding>true</EnableCOMDATFolding>
-      <OptimizeReferences>true</OptimizeReferences>
-    </Link>
+    <CustomBuildStep>
+      <Message>Compiling Messages</Message>
+      <Command>mc.exe $(TargetName).mc -z $(TargetName)_msg -r $(IntermediateOutputPath) -h $(IntermediateOutputPath) -U</Command>
+      <Outputs>$(IntermediateOutputPath)$(TargetName)_msg.rc;$(IntermediateOutputPath)$(TargetName)_msg.h</Outputs>
+    </CustomBuildStep>
+    <Midl>
+      <ApplicationConfigurationMode>true</ApplicationConfigurationMode>
+      <TargetEnvironment>X64</TargetEnvironment>
+      <OutputDirectory>$(IntermediateOutputPath)</OutputDirectory>
+      <GenerateStublessProxies>true</GenerateStublessProxies>
+      <ValidateAllParameters>true</ValidateAllParameters>
+      <WarnAsError>true</WarnAsError>
+      <WarningLevel>2</WarningLevel>
+    </Midl>
   </ItemDefinitionGroup>
+  <PropertyGroup>
+    <CustomBuildAfterTargets>Midl</CustomBuildAfterTargets>
+    <CustomBuildBeforeTargets>ClCompile,ResourceCompile</CustomBuildBeforeTargets>
+  </PropertyGroup>
+  <ItemGroup>
+    <Midl Include="hadoopwinutilsvc.idl" />
+  </ItemGroup>
+  <ItemGroup>
+    <ResourceCompile Include="$(IntermediateOutputPath)$(TargetName)_msg.rc" />
+  </ItemGroup>
+  <ItemGroup>
+    <ClCompile Include="$(IntermediateOutputPath)\hadoopwinutilsvc_s.c" />
+  </ItemGroup>
   <ItemGroup>
+    <ClCompile Include="service.c" />
     <ClCompile Include="readlink.c" />
     <ClCompile Include="symlink.c" />
     <ClCompile Include="systeminfo.c" />
@@ -179,4 +145,4 @@
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />
   <ImportGroup Label="ExtensionTargets">
   </ImportGroup>
-</Project>
+</Project>

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ProcessTree.java

@@ -296,7 +296,7 @@ public class ProcessTree {
       return false;
     } catch (IOException ioe) {
       LOG.warn("Error executing shell command "
-          + Arrays.toString(shexec.getExecString()) + ioe);
+          + shexec.toString() + ioe);
       return false;
     }
     return (shexec.getExitCode() == 0 ? true : false);
@@ -321,7 +321,7 @@ public class ProcessTree {
       return false;
     } catch (IOException ioe) {
       LOG.warn("Error executing shell command "
-          + Arrays.toString(shexec.getExecString()) + ioe);
+          + shexec.toString() + ioe);
       return false;
     }
     return (shexec.getExitCode() == 0 ? true : false);

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -178,6 +178,9 @@ Release 2.6.0 - UNRELEASED
     YARN-2504. Enhanced RM Admin CLI to support management of node-labels.
     (Wangda Tan via vinodkv)
 
+    YARN-2198. Remove the need to run NodeManager as privileged account for
+    Windows Secure Container Executor. (Remus Rusanu via jianhe)
+
   IMPROVEMENTS
 
     YARN-2197. Add a link to YARN CHANGES.txt in the left side of doc

+ 14 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java

@@ -79,19 +79,23 @@ public abstract class ContainerExecutor implements Configurable {
   public abstract void init() throws IOException;
 
   /**
-   * On Windows the ContainerLaunch creates a temporary empty jar to workaround the CLASSPATH length
-   * In a  secure cluster this jar must be localized so that the container has access to it
+   * On Windows the ContainerLaunch creates a temporary special jar manifest of 
+   * other jars to workaround the CLASSPATH length. In a  secure cluster this 
+   * jar must be localized so that the container has access to it. 
    * This function localizes on-demand the jar.
    * 
    * @param classPathJar
    * @param owner
    * @throws IOException
    */
-  public void localizeClasspathJar(Path classPathJar, String owner) throws IOException {
-    // For the default container this is a no-op
-    // The WindowsSecureContainerExecutor overrides this
+  public Path localizeClasspathJar(Path classPathJar, Path pwd, String owner) 
+      throws IOException {
+    // Non-secure executor simply use the classpath created 
+    // in the NM fprivate folder
+    return classPathJar;
   }
-
+  
+  
   /**
    * Prepare the environment for containers in this application to execute.
    * For $x in local.dirs
@@ -105,14 +109,13 @@ public abstract class ContainerExecutor implements Configurable {
    * @param appId id of the application
    * @param nmPrivateContainerTokens path to localized credentials, rsrc by NM
    * @param nmAddr RPC address to contact NM
-   * @param localDirs nm-local-dirs
-   * @param logDirs nm-log-dirs
+   * @param dirsHandler NM local dirs service, for nm-local-dirs and nm-log-dirs
    * @throws IOException For most application init failures
    * @throws InterruptedException If application init thread is halted by NM
    */
   public abstract void startLocalizer(Path nmPrivateContainerTokens,
       InetSocketAddress nmAddr, String user, String appId, String locId,
-      List<String> localDirs, List<String> logDirs)
+      LocalDirsHandlerService dirsHandler)
     throws IOException, InterruptedException;
 
 
@@ -132,8 +135,8 @@ public abstract class ContainerExecutor implements Configurable {
    */
   public abstract int launchContainer(Container container,
       Path nmPrivateContainerScriptPath, Path nmPrivateTokensPath,
-      String user, String appId, Path containerWorkDir, List<String> localDirs,
-      List<String> logDirs) throws IOException;
+      String user, String appId, Path containerWorkDir, 
+      List<String> localDirs, List<String> logDirs) throws IOException;
 
   public abstract boolean signalContainer(String user, String pid,
       Signal signal)

+ 46 - 27
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DefaultContainerExecutor.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.server.nodemanager;
 
 import com.google.common.base.Optional;
+
 import static org.apache.hadoop.fs.CreateFlag.CREATE;
 import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
 
@@ -32,9 +33,11 @@ import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Random;
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
@@ -42,6 +45,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell.ExitCodeException;
+import org.apache.hadoop.util.Shell.CommandExecutor;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -92,13 +96,16 @@ public class DefaultContainerExecutor extends ContainerExecutor {
   @Override
   public synchronized void startLocalizer(Path nmPrivateContainerTokensPath,
       InetSocketAddress nmAddr, String user, String appId, String locId,
-      List<String> localDirs, List<String> logDirs)
+      LocalDirsHandlerService dirsHandler)
       throws IOException, InterruptedException {
 
+    List<String> localDirs = dirsHandler.getLocalDirs();
+    List<String> logDirs = dirsHandler.getLogDirs();
+    
     ContainerLocalizer localizer =
         new ContainerLocalizer(lfs, user, appId, locId, getPaths(localDirs),
             RecordFactoryProvider.getRecordFactory(getConf()));
-
+    
     createUserLocalDirs(localDirs, user);
     createUserCacheDirs(localDirs, user);
     createAppDirs(localDirs, user, appId);
@@ -120,9 +127,9 @@ public class DefaultContainerExecutor extends ContainerExecutor {
   @Override
   public int launchContainer(Container container,
       Path nmPrivateContainerScriptPath, Path nmPrivateTokensPath,
-      String userName, String appId, Path containerWorkDir,
+      String user, String appId, Path containerWorkDir,
       List<String> localDirs, List<String> logDirs) throws IOException {
-
+    
     FsPermission dirPerm = new FsPermission(APPDIR_PERM);
     ContainerId containerId = container.getContainerId();
 
@@ -134,29 +141,30 @@ public class DefaultContainerExecutor extends ContainerExecutor {
                 getApplicationId());
     for (String sLocalDir : localDirs) {
       Path usersdir = new Path(sLocalDir, ContainerLocalizer.USERCACHE);
-      Path userdir = new Path(usersdir, userName);
+      Path userdir = new Path(usersdir, user);
       Path appCacheDir = new Path(userdir, ContainerLocalizer.APPCACHE);
       Path appDir = new Path(appCacheDir, appIdStr);
       Path containerDir = new Path(appDir, containerIdStr);
-      createDir(containerDir, dirPerm, true, userName);
+      createDir(containerDir, dirPerm, true, user);
     }
 
     // Create the container log-dirs on all disks
-    createContainerLogDirs(appIdStr, containerIdStr, logDirs, userName);
+    createContainerLogDirs(appIdStr, containerIdStr, logDirs, user);
 
     Path tmpDir = new Path(containerWorkDir,
         YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR);
-    createDir(tmpDir, dirPerm, false, userName);
+    createDir(tmpDir, dirPerm, false, user);
 
-    // copy launch script to work dir
-    Path launchDst =
-        new Path(containerWorkDir, ContainerLaunch.CONTAINER_SCRIPT);
-    copyFile(nmPrivateContainerScriptPath, launchDst, userName);
 
     // copy container tokens to work dir
     Path tokenDst =
       new Path(containerWorkDir, ContainerLaunch.FINAL_CONTAINER_TOKENS_FILE);
-    copyFile(nmPrivateTokensPath, tokenDst, userName);
+    copyFile(nmPrivateTokensPath, tokenDst, user);
+
+    // copy launch script to work dir
+    Path launchDst =
+        new Path(containerWorkDir, ContainerLaunch.CONTAINER_SCRIPT);
+    copyFile(nmPrivateContainerScriptPath, launchDst, user);
 
     // Create new local launch wrapper script
     LocalWrapperScriptBuilder sb = getLocalWrapperScriptBuilder(
@@ -181,23 +189,19 @@ public class DefaultContainerExecutor extends ContainerExecutor {
           + " was marked as inactive. Returning terminated error");
       return ExitCode.TERMINATED.getExitCode();
     }
-
+    
     // create log dir under app
     // fork script
-    ShellCommandExecutor shExec = null;
+    Shell.CommandExecutor shExec = null;
     try {
-      setScriptExecutable(launchDst, userName);
-      setScriptExecutable(sb.getWrapperScriptPath(), userName);
-
-      // Setup command to run
-      String[] command = getRunCommand(sb.getWrapperScriptPath().toString(),
-        containerIdStr, userName, pidFile, this.getConf());
+      setScriptExecutable(launchDst, user);
+      setScriptExecutable(sb.getWrapperScriptPath(), user);
 
-      LOG.info("launchContainer: " + Arrays.toString(command));
-      shExec = new ShellCommandExecutor(
-          command,
+      shExec = buildCommandExecutor(sb.getWrapperScriptPath().toString(),
+          containerIdStr, user, pidFile,
           new File(containerWorkDir.toUri().getPath()),
-          container.getLaunchContext().getEnvironment());      // sanitized env
+          container.getLaunchContext().getEnvironment());
+      
       if (isContainerActive(containerId)) {
         shExec.execute();
       }
@@ -242,11 +246,26 @@ public class DefaultContainerExecutor extends ContainerExecutor {
       }
       return exitCode;
     } finally {
-      ; //
+      if (shExec != null) shExec.close();
     }
     return 0;
   }
 
+  protected CommandExecutor buildCommandExecutor(String wrapperScriptPath, 
+      String containerIdStr, String user, Path pidFile, File wordDir, 
+      Map<String, String> environment) 
+          throws IOException {
+    
+    String[] command = getRunCommand(wrapperScriptPath,
+        containerIdStr, user, pidFile, this.getConf());
+
+      LOG.info("launchContainer: " + Arrays.toString(command));
+      return new ShellCommandExecutor(
+          command,
+          wordDir,
+          environment); 
+  }
+
   protected LocalWrapperScriptBuilder getLocalWrapperScriptBuilder(
       String containerIdStr, Path containerWorkDir) {
    return  Shell.WINDOWS ?
@@ -421,7 +440,7 @@ public class DefaultContainerExecutor extends ContainerExecutor {
    * @param signal signal to send
    * (for logging).
    */
-  private void killContainer(String pid, Signal signal) throws IOException {
+  protected void killContainer(String pid, Signal signal) throws IOException {
     new ShellCommandExecutor(Shell.getSignalKillCommand(signal.getValue(), pid))
       .execute();
   }

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java

@@ -196,9 +196,12 @@ public class LinuxContainerExecutor extends ContainerExecutor {
   @Override
   public void startLocalizer(Path nmPrivateContainerTokensPath,
       InetSocketAddress nmAddr, String user, String appId, String locId,
-      List<String> localDirs, List<String> logDirs)
+      LocalDirsHandlerService dirsHandler)
       throws IOException, InterruptedException {
 
+    List<String> localDirs = dirsHandler.getLocalDirs();
+    List<String> logDirs = dirsHandler.getLogDirs();
+    
     verifyUsernamePattern(user);
     String runAsUser = getRunAsUser(user);
     List<String> command = new ArrayList<String>();

+ 606 - 47
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java

@@ -17,36 +17,272 @@
  */
 package org.apache.hadoop.yarn.server.nodemanager;
 
+import java.io.BufferedReader;
 import java.io.File;
+import java.io.FileDescriptor;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
 import java.io.PrintStream;
 import java.net.InetSocketAddress;
+import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DelegateToFileSystem;
+import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.nativeio.NativeIO.Windows;
+import org.apache.hadoop.io.nativeio.NativeIOException;
+import org.apache.hadoop.util.NativeCodeLoader;
 import org.apache.hadoop.util.Shell;
-import org.apache.hadoop.util.Shell.ShellCommandExecutor;
+import org.apache.hadoop.util.Shell.CommandExecutor;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.LocalWrapperScriptBuilder;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
 
 /**
- * Windows secure container executor. Uses winutils task createAsUser.
- *
+ * Windows secure container executor (WSCE).
+ * This class offers a secure container executor on Windows, similar to the 
+ * LinuxContainerExecutor. As the NM does not run on a high privileged context, 
+ * this class delegates elevated operations to the helper hadoopwintuilsvc, 
+ * implemented by the winutils.exe running as a service.
+ * JNI and LRPC is used to communicate with the privileged service.
  */
 public class WindowsSecureContainerExecutor extends DefaultContainerExecutor {
-
+  
   private static final Log LOG = LogFactory
       .getLog(WindowsSecureContainerExecutor.class);
+  
+  public static final String LOCALIZER_PID_FORMAT = "STAR_LOCALIZER_%s";
+  
+  
+  /**
+   * This class is a container for the JNI Win32 native methods used by WSCE.
+   */
+  private static class Native {
+
+    private static boolean nativeLoaded = false;
+
+    static {
+      if (NativeCodeLoader.isNativeCodeLoaded()) {
+        try {
+          initWsceNative();
+          nativeLoaded = true;
+        } catch (Throwable t) {
+          LOG.info("Unable to initialize WSCE Native libraries", t);
+        }
+      }
+    }
+
+    /** Initialize the JNI method ID and class ID cache */
+    private static native void initWsceNative();
+    
+    
+    /**
+     * This class contains methods used by the WindowsSecureContainerExecutor
+     * file system operations.
+     */
+    public static class Elevated {
+      private static final int MOVE_FILE = 1;
+      private static final int COPY_FILE = 2;
+
+      public static void mkdir(Path dirName) throws IOException {
+        if (!nativeLoaded) {
+          throw new IOException("Native WSCE libraries are required for mkdir");
+        }
+        elevatedMkDirImpl(dirName.toString());
+      }
+      
+      private static native void elevatedMkDirImpl(String dirName) 
+          throws IOException;
+      
+      public static void chown(Path fileName, String user, String group) 
+          throws IOException {
+        if (!nativeLoaded) {
+          throw new IOException("Native WSCE libraries are required for chown");
+        }
+        elevatedChownImpl(fileName.toString(), user, group);
+      }
+      
+      private static native void elevatedChownImpl(String fileName, String user, 
+          String group) throws IOException;
+      
+      public static void move(Path src, Path dst, boolean replaceExisting) 
+          throws IOException {
+        if (!nativeLoaded) {
+          throw new IOException("Native WSCE libraries are required for move");
+        }
+        elevatedCopyImpl(MOVE_FILE, src.toString(), dst.toString(), 
+            replaceExisting);
+      }
+      
+      public static void copy(Path src, Path dst, boolean replaceExisting) 
+          throws IOException {
+        if (!nativeLoaded) {
+          throw new IOException("Native WSCE libraries are required for copy");
+        }
+        elevatedCopyImpl(COPY_FILE, src.toString(), dst.toString(), 
+            replaceExisting);
+      }
+      
+      private static native void elevatedCopyImpl(int operation, String src, 
+          String dst, boolean replaceExisting) throws IOException;
+      
+      public static void chmod(Path fileName, int mode) throws IOException {
+        if (!nativeLoaded) {
+          throw new IOException("Native WSCE libraries are required for chmod");
+        }
+        elevatedChmodImpl(fileName.toString(), mode);
+      }
+      
+      private static native void elevatedChmodImpl(String path, int mode) 
+          throws IOException;
+      
+      public static void killTask(String containerName) throws IOException {
+        if (!nativeLoaded) {
+          throw new IOException("Native WSCE libraries are required for killTask");
+        }
+        elevatedKillTaskImpl(containerName);
+      }
+      
+      private static native void elevatedKillTaskImpl(String containerName) 
+          throws IOException;
+
+      public static OutputStream create(Path f, boolean append)  
+          throws IOException {
+        if (!nativeLoaded) {
+          throw new IOException("Native WSCE libraries are required for create");
+        }
+        
+        long desiredAccess = Windows.GENERIC_WRITE;
+        long shareMode = 0L;
+        long creationDisposition = append ? 
+            Windows.OPEN_ALWAYS : Windows.CREATE_ALWAYS;
+        long flags = Windows.FILE_ATTRIBUTE_NORMAL;
+        
+        String fileName = f.toString();
+        fileName = fileName.replace('/', '\\');
+        
+        long hFile = elevatedCreateImpl(
+            fileName, desiredAccess, shareMode, creationDisposition, flags);
+        return new FileOutputStream(
+            WinutilsProcessStub.getFileDescriptorFromHandle(hFile));
+      }
+      
+      private static native long elevatedCreateImpl(String path, 
+          long desiredAccess, long shareMode,
+          long creationDisposition, long flags) throws IOException;
+      
+      
+      public static boolean deleteFile(Path path) throws IOException {
+        if (!nativeLoaded) {
+          throw new IOException("Native WSCE libraries are required for deleteFile");
+        }
+        
+        return elevatedDeletePathImpl(path.toString(), false);
+      }
+
+      public static boolean deleteDirectory(Path path) throws IOException {
+        if (!nativeLoaded) {
+          throw new IOException("Native WSCE libraries are required for deleteDirectory");
+        }
+        
+        return elevatedDeletePathImpl(path.toString(), true);
+      }
+
+      public native static boolean elevatedDeletePathImpl(String path, 
+          boolean isDir) throws IOException;
+    }
+
+    /**
+     * Wraps a process started by the winutils service helper.
+     *
+     */
+    public static class WinutilsProcessStub extends Process {
+      
+      private final long hProcess;
+      private final long hThread;
+      private boolean disposed = false;
+      
+      private final InputStream stdErr;
+      private final InputStream stdOut;
+      private final OutputStream stdIn;
+      
+      public WinutilsProcessStub(long hProcess, long hThread, long hStdIn, 
+          long hStdOut, long hStdErr) {
+        this.hProcess = hProcess;
+        this.hThread = hThread;
+        
+        this.stdIn = new FileOutputStream(getFileDescriptorFromHandle(hStdIn));
+        this.stdOut = new FileInputStream(getFileDescriptorFromHandle(hStdOut));
+        this.stdErr = new FileInputStream(getFileDescriptorFromHandle(hStdErr));
+      }
+      
+      public static native FileDescriptor getFileDescriptorFromHandle(long handle);
+      
+      @Override
+      public native void destroy();
+      
+      @Override
+      public native int exitValue();
+      
+      @Override
+      public InputStream getErrorStream() {
+        return stdErr;
+      }
+      @Override
+      public InputStream getInputStream() {
+        return stdOut;
+      }
+      @Override
+      public OutputStream getOutputStream() {
+        return stdIn;
+      }
+      @Override
+      public native int waitFor() throws InterruptedException;
+
+      public synchronized native void dispose();
+
+      public native void resume() throws NativeIOException;
+    }
+    
+    public synchronized static WinutilsProcessStub createTaskAsUser(
+        String cwd, String jobName, String user, String pidFile, String cmdLine)
+      throws IOException {
+      if (!nativeLoaded) {
+        throw new IOException(
+            "Native WSCE  libraries are required for createTaskAsUser");
+      }
+      synchronized(Shell.WindowsProcessLaunchLock) {
+        return createTaskAsUser0(cwd, jobName, user, pidFile, cmdLine);
+      }
+    }
 
+    private static native WinutilsProcessStub createTaskAsUser0(
+      String cwd, String jobName, String user, String pidFile, String cmdLine)
+      throws NativeIOException;
+  }
+
+  /**
+   * A shell script wrapper builder for WSCE.  
+   * Overwrites the default behavior to remove the creation of the PID file in 
+   * the script wrapper. WSCE creates the pid file as part of launching the 
+   * task in winutils.
+   */
   private class WindowsSecureWrapperScriptBuilder 
     extends LocalWrapperScriptBuilder {
 
@@ -60,21 +296,278 @@ public class WindowsSecureContainerExecutor extends DefaultContainerExecutor {
     }
   }
 
+  /**
+   * This is a skeleton file system used to elevate certain operations.
+   * WSCE has to create container dirs under local/userchache/$user but
+   * this dir itself is owned by $user, with chmod 750. As ther NM has no
+   * write access, it must delegate the write operations to the privileged
+   * hadoopwintuilsvc.
+   */
+  private static class ElevatedFileSystem extends DelegateToFileSystem {
+
+    /**
+     * This overwrites certain RawLocalSystem operations to be performed by a 
+     * privileged process.
+     * 
+     */
+    private static class ElevatedRawLocalFilesystem extends RawLocalFileSystem {
+      
+      @Override
+      protected boolean mkOneDir(File p2f) throws IOException {
+        Path path = new Path(p2f.getAbsolutePath());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(String.format("EFS:mkOneDir: %s", path));
+        }
+        boolean ret = false;
+
+        // File.mkdir returns false, does not throw. Must mimic it.
+        try {
+          Native.Elevated.mkdir(path);
+          ret = true;
+        }
+        catch(Throwable e) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(String.format("EFS:mkOneDir: %s", 
+                org.apache.hadoop.util.StringUtils.stringifyException(e)));
+          }
+        }
+        return ret;
+      }
+      
+      @Override
+      public void setPermission(Path p, FsPermission permission) 
+          throws IOException {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(String.format("EFS:setPermission: %s %s", p, permission));
+        }
+        Native.Elevated.chmod(p, permission.toShort());
+      }
+      
+      @Override
+      public void setOwner(Path p, String username, String groupname) 
+          throws IOException {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(String.format("EFS:setOwner: %s %s %s", 
+              p, username, groupname));
+        }
+        Native.Elevated.chown(p, username, groupname);
+      }
+      
+      @Override
+      protected OutputStream createOutputStream(Path f, boolean append) 
+          throws IOException {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(String.format("EFS:create: %s %b", f, append));
+        }
+        return Native.Elevated.create(f, append); 
+      }
+      
+      @Override
+      public boolean delete(Path p, boolean recursive) throws IOException {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(String.format("EFS:delete: %s %b", p, recursive));
+        }
+        
+        // The super delete uses the FileUtil.fullyDelete, 
+        // but we cannot rely on that because we need to use the elevated 
+        // operations to remove the files
+        //
+        File f = pathToFile(p);
+        if (!f.exists()) {
+          //no path, return false "nothing to delete"
+          return false;
+        }
+        else if (f.isFile()) {
+          return Native.Elevated.deleteFile(p);
+        } 
+        else if (f.isDirectory()) {
+          
+          // This is a best-effort attempt. There are race conditions in that
+          // child files can be created/deleted after we snapped the list. 
+          // No need to protect against that case.
+          File[] files = FileUtil.listFiles(f);
+          int childCount = files.length;
+          
+          if (recursive) {
+            for(File child:files) {
+              if (delete(new Path(child.getPath()), recursive)) {
+                --childCount;
+              }
+            }
+          }
+          if (childCount == 0) {
+            return Native.Elevated.deleteDirectory(p);
+          } 
+          else {
+            throw new IOException("Directory " + f.toString() + " is not empty");
+          }
+        }
+        else {
+          // This can happen under race conditions if an external agent 
+          // is messing with the file type between IFs
+          throw new IOException("Path " + f.toString() + 
+              " exists, but is neither a file nor a directory");
+        }
+      }
+    }
+
+    protected ElevatedFileSystem() throws IOException, URISyntaxException {
+      super(FsConstants.LOCAL_FS_URI,
+          new ElevatedRawLocalFilesystem(), 
+          new Configuration(),
+          FsConstants.LOCAL_FS_URI.getScheme(),
+          false);
+    }
+  }
+  
+  private static class WintuilsProcessStubExecutor 
+  implements Shell.CommandExecutor {
+    private Native.WinutilsProcessStub processStub;
+    private StringBuilder output = new StringBuilder();
+    private int exitCode;
+    
+    private enum State {
+      INIT,
+      RUNNING,
+      COMPLETE
+    };
+    
+    private State state;;
+    
+    private final String cwd;
+    private final String jobName;
+    private final String userName;
+    private final String pidFile;
+    private final String cmdLine;
+
+    public WintuilsProcessStubExecutor(
+        String cwd, 
+        String jobName, 
+        String userName, 
+        String pidFile,
+        String cmdLine) {
+      this.cwd = cwd;
+      this.jobName = jobName;
+      this.userName = userName;
+      this.pidFile = pidFile;
+      this.cmdLine = cmdLine;
+      this.state = State.INIT;
+    }    
+    
+    private void assertComplete() throws IOException {
+      if (state != State.COMPLETE) {
+        throw new IOException("Process is not complete");
+      }
+    }
+    
+    public String getOutput () throws IOException {
+      assertComplete();
+      return output.toString();
+    }
+    
+    public int getExitCode() throws IOException {
+      assertComplete();
+      return exitCode;
+    }
+    
+    public void validateResult() throws IOException {
+      assertComplete();
+      if (0 != exitCode) {
+        LOG.warn(output.toString());
+        throw new IOException("Processs exit code is:" + exitCode);
+      }
+    }
+    
+    private Thread startStreamReader(final InputStream stream) 
+        throws IOException {
+      Thread streamReaderThread = new Thread() {
+        
+        @Override
+        public void run() {
+          try
+          {
+            BufferedReader lines = new BufferedReader(
+                new InputStreamReader(stream));
+            char[] buf = new char[512];
+            int nRead;
+            while ((nRead = lines.read(buf, 0, buf.length)) > 0) {
+              output.append(buf, 0, nRead);
+            }
+          }
+          catch(Throwable t) {
+            LOG.error("Error occured reading the process stdout", t);
+          }
+        }
+      };
+      streamReaderThread.start();
+      return streamReaderThread;
+    }
+
+    public void execute() throws IOException {
+      if (state != State.INIT) {
+        throw new IOException("Process is already started");
+      }
+      processStub = Native.createTaskAsUser(cwd,
+          jobName, userName, pidFile, cmdLine);
+      state = State.RUNNING;
+
+      Thread stdOutReader = startStreamReader(processStub.getInputStream());
+      Thread stdErrReader = startStreamReader(processStub.getErrorStream());
+      
+      try {
+        processStub.resume();
+        processStub.waitFor();
+        stdOutReader.join();
+        stdErrReader.join();
+      }
+      catch(InterruptedException ie) {
+        throw new IOException(ie);
+      }
+      
+      exitCode = processStub.exitValue();
+      state = State.COMPLETE;
+    }
+
+    @Override
+    public void close() {
+      if (processStub != null) {
+        processStub.dispose();
+      }
+    }
+  }
+
   private String nodeManagerGroup;
+  
+  /** 
+   * Permissions for user WSCE dirs.
+   */
+  static final short DIR_PERM = (short)0750;  
+  
+  public WindowsSecureContainerExecutor() 
+      throws IOException, URISyntaxException {
+    super(FileContext.getFileContext(new ElevatedFileSystem(), 
+        new Configuration()));
+  }
 
   @Override
   public void setConf(Configuration conf) {
     super.setConf(conf);
-    nodeManagerGroup = conf.get(YarnConfiguration.NM_WINDOWS_SECURE_CONTAINER_GROUP);
+    nodeManagerGroup = conf.get(
+        YarnConfiguration.NM_WINDOWS_SECURE_CONTAINER_GROUP);
   }
-
+  
   @Override
   protected String[] getRunCommand(String command, String groupId,
       String userName, Path pidFile, Configuration conf) {
-    return new String[] { Shell.WINUTILS, "task", "createAsUser", groupId, userName,
-        pidFile.toString(), "cmd /c " + command };
+    File f = new File(command);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(String.format("getRunCommand: %s exists:%b", 
+          command, f.exists()));
+    }
+    return new String[] { Shell.WINUTILS, "task", "createAsUser", groupId, 
+        userName, pidFile.toString(), "cmd /c " + command };
   }
-
+  
   @Override
   protected LocalWrapperScriptBuilder getLocalWrapperScriptBuilder(
       String containerIdStr, Path containerWorkDir) {
@@ -83,90 +576,156 @@ public class WindowsSecureContainerExecutor extends DefaultContainerExecutor {
   
   @Override
   protected void copyFile(Path src, Path dst, String owner) throws IOException {
-    super.copyFile(src, dst, owner);
-    lfs.setOwner(dst,  owner, nodeManagerGroup);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(String.format("copyFile: %s -> %s owner:%s", src.toString(), 
+          dst.toString(), owner));
+    }
+    Native.Elevated.copy(src,  dst, true);
+    Native.Elevated.chown(dst, owner, nodeManagerGroup);
   }
 
   @Override
   protected void createDir(Path dirPath, FsPermission perms,
       boolean createParent, String owner) throws IOException {
+    
+    // WSCE requires dirs to be 750, not 710 as DCE.
+    // This is similar to how LCE creates dirs
+    //
+    perms = new FsPermission(DIR_PERM);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(String.format("createDir: %s perm:%s owner:%s", 
+          dirPath.toString(), perms.toString(), owner));
+    }
+    
     super.createDir(dirPath, perms, createParent, owner);
     lfs.setOwner(dirPath, owner, nodeManagerGroup);
   }
 
   @Override
-  protected void setScriptExecutable(Path script, String owner) throws IOException {
-    super.setScriptExecutable(script, null);
-    lfs.setOwner(script, owner, nodeManagerGroup);
+  protected void setScriptExecutable(Path script, String owner) 
+      throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(String.format("setScriptExecutable: %s owner:%s", 
+          script.toString(), owner));
+    }
+    super.setScriptExecutable(script, owner);
+    Native.Elevated.chown(script, owner, nodeManagerGroup);
   }
 
   @Override
-  public void localizeClasspathJar(Path classpathJar, String owner) throws IOException {
-    lfs.setOwner(classpathJar, owner, nodeManagerGroup);
+  public Path localizeClasspathJar(Path classPathJar, Path pwd, String owner) 
+      throws IOException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(String.format("localizeClasspathJar: %s %s o:%s", 
+          classPathJar, pwd, owner));
+    }
+    createDir(pwd,  new FsPermission(DIR_PERM), true, owner);
+    String fileName = classPathJar.getName();
+    Path dst = new Path(pwd, fileName);
+    Native.Elevated.move(classPathJar, dst, true);
+    Native.Elevated.chown(dst, owner, nodeManagerGroup);
+    return dst;
   }
 
  @Override
  public void startLocalizer(Path nmPrivateContainerTokens,
      InetSocketAddress nmAddr, String user, String appId, String locId,
-     List<String> localDirs, List<String> logDirs) throws IOException,
+     LocalDirsHandlerService dirsHandler) throws IOException,
      InterruptedException {
-
+   
+     List<String> localDirs = dirsHandler.getLocalDirs();
+     List<String> logDirs = dirsHandler.getLogDirs();
+     
+     Path classpathJarPrivateDir = dirsHandler.getLocalPathForWrite(
+         ResourceLocalizationService.NM_PRIVATE_DIR);
      createUserLocalDirs(localDirs, user);
      createUserCacheDirs(localDirs, user);
      createAppDirs(localDirs, user, appId);
      createAppLogDirs(appId, logDirs, user);
 
      Path appStorageDir = getWorkingDir(localDirs, user, appId);
-
-     String tokenFn = String.format(ContainerLocalizer.TOKEN_FILE_NAME_FMT, locId);
+     
+     String tokenFn = String.format(
+         ContainerLocalizer.TOKEN_FILE_NAME_FMT, locId);
      Path tokenDst = new Path(appStorageDir, tokenFn);
-     LOG.info("Copying from " + nmPrivateContainerTokens + " to " + tokenDst);
      copyFile(nmPrivateContainerTokens, tokenDst, user);
 
-     List<String> command ;
-     String[] commandArray;
-     ShellCommandExecutor shExec;
-
      File cwdApp = new File(appStorageDir.toString());
-     LOG.info(String.format("cwdApp: %s", cwdApp));
+     if (LOG.isDebugEnabled()) {
+       LOG.debug(String.format("cwdApp: %s", cwdApp));
+     }
+     
+     List<String> command ;
 
      command = new ArrayList<String>();
 
-     command.add(Shell.WINUTILS);
-     command.add("task");
-     command.add("createAsUser");
-     command.add("START_LOCALIZER_" + locId);
-     command.add(user);
-     command.add("nul:"); // PID file    
-   
    //use same jvm as parent
-     File jvm = new File(new File(System.getProperty("java.home"), "bin"), "java.exe");
+     File jvm = new File(
+         new File(System.getProperty("java.home"), "bin"), "java.exe");
      command.add(jvm.toString());
      
+     Path cwdPath = new Path(cwdApp.getPath());
      
      // Build a temp classpath jar. See ContainerLaunch.sanitizeEnv().
      // Passing CLASSPATH explicitly is *way* too long for command line.
      String classPath = System.getProperty("java.class.path");
      Map<String, String> env = new HashMap<String, String>(System.getenv());
-     String[] jarCp = FileUtil.createJarWithClassPath(classPath, appStorageDir, env);
-     String classPathJar = jarCp[0];
-     localizeClasspathJar(new Path(classPathJar), user);
-     String replacementClassPath = classPathJar + jarCp[1];
+     String jarCp[] = FileUtil.createJarWithClassPath(classPath, 
+         classpathJarPrivateDir, cwdPath, env);
+     String classPathJar = localizeClasspathJar(
+         new Path(jarCp[0]), cwdPath, user).toString();
      command.add("-classpath");
-     command.add(replacementClassPath);
-     
+     command.add(classPathJar + jarCp[1]);
+
      String javaLibPath = System.getProperty("java.library.path");
      if (javaLibPath != null) {
        command.add("-Djava.library.path=" + javaLibPath);
      }
      
-     ContainerLocalizer.buildMainArgs(command, user, appId, locId, nmAddr, localDirs);
-     commandArray = command.toArray(new String[command.size()]);
-
-     shExec = new ShellCommandExecutor(
-         commandArray, cwdApp);
+     ContainerLocalizer.buildMainArgs(command, user, appId, locId, nmAddr, 
+         localDirs);
+     
+     String cmdLine = StringUtils.join(command, " ");
+     
+     String localizerPid = String.format(LOCALIZER_PID_FORMAT, locId);
+     
+     WintuilsProcessStubExecutor stubExecutor = new WintuilsProcessStubExecutor(
+         cwdApp.getAbsolutePath(), 
+         localizerPid, user, "nul:", cmdLine);
+     try {
+       stubExecutor.execute();
+       stubExecutor.validateResult();
+     }
+     finally {
+       stubExecutor.close();
+       try
+       {
+         killContainer(localizerPid, Signal.KILL);
+       }
+       catch(Throwable e) {
+         LOG.warn(String.format(
+             "An exception occured during the cleanup of localizer job %s:\n%s", 
+             localizerPid, 
+             org.apache.hadoop.util.StringUtils.stringifyException(e)));
+       }
+     }
+   }
+ 
+   @Override
+   protected CommandExecutor buildCommandExecutor(String wrapperScriptPath, 
+       String containerIdStr,
+     String userName, Path pidFile,File wordDir, Map<String, String> environment) 
+     throws IOException {
 
-     shExec.execute();
+     return new WintuilsProcessStubExecutor(
+         wordDir.toString(),
+         containerIdStr, userName, pidFile.toString(), 
+         "cmd /c " + wrapperScriptPath);
+   }
+   
+   @Override
+   protected void killContainer(String pid, Signal signal) throws IOException {
+     Native.Elevated.killTask(pid);
    }
 }
 

+ 12 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java

@@ -212,7 +212,9 @@ public class ContainerLaunch implements Callable<Integer> {
                   + Path.SEPARATOR
                   + String.format(ContainerLocalizer.TOKEN_FILE_NAME_FMT,
                       containerIdStr));
-
+      Path nmPrivateClasspathJarDir = 
+          dirsHandler.getLocalPathForWrite(
+              getContainerPrivateDir(appIdStr, containerIdStr));
       DataOutputStream containerScriptOutStream = null;
       DataOutputStream tokensOutStream = null;
 
@@ -263,7 +265,7 @@ public class ContainerLaunch implements Callable<Integer> {
                 FINAL_CONTAINER_TOKENS_FILE).toUri().getPath());
         // Sanitize the container's environment
         sanitizeEnv(environment, containerWorkDir, appDirs, containerLogDirs,
-          localResources);
+          localResources, nmPrivateClasspathJarDir);
         
         // Write out the environment
         writeLaunchEnv(containerScriptOutStream, environment, localResources,
@@ -658,7 +660,8 @@ public class ContainerLaunch implements Callable<Integer> {
   
   public void sanitizeEnv(Map<String, String> environment, Path pwd,
       List<Path> appDirs, List<String> containerLogDirs,
-      Map<Path, List<String>> resources) throws IOException {
+      Map<Path, List<String>> resources,
+      Path nmPrivateClasspathJarDir) throws IOException {
     /**
      * Non-modifiable environment variables
      */
@@ -722,6 +725,7 @@ public class ContainerLaunch implements Callable<Integer> {
     // TODO: Remove Windows check and use this approach on all platforms after
     // additional testing.  See YARN-358.
     if (Shell.WINDOWS) {
+      
       String inputClassPath = environment.get(Environment.CLASSPATH.name());
       if (inputClassPath != null && !inputClassPath.isEmpty()) {
         StringBuilder newClassPath = new StringBuilder(inputClassPath);
@@ -763,13 +767,13 @@ public class ContainerLaunch implements Callable<Integer> {
         Map<String, String> mergedEnv = new HashMap<String, String>(
           System.getenv());
         mergedEnv.putAll(environment);
-
+        
         String[] jarCp = FileUtil.createJarWithClassPath(
-          newClassPath.toString(), pwd, mergedEnv);
-        String classPathJar = jarCp[0];
+          newClassPath.toString(), nmPrivateClasspathJarDir, pwd, mergedEnv);
         // In a secure cluster the classpath jar must be localized to grant access
-        this.exec.localizeClasspathJar(new Path(classPathJar), container.getUser());
-        String replacementClassPath = classPathJar + jarCp[1];
+        Path localizedClassPathJar = exec.localizeClasspathJar(
+            new Path(jarCp[0]), pwd, container.getUser());
+        String replacementClassPath = localizedClassPathJar.toString() + jarCp[1];
         environment.put(Environment.CLASSPATH.name(), replacementClassPath);
       }
     }

+ 6 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java

@@ -369,10 +369,15 @@ public class ContainerLocalizer {
           new ContainerLocalizer(FileContext.getLocalFSFileContext(), user,
               appId, locId, localDirs,
               RecordFactoryProvider.getRecordFactory(null));
-      System.exit(localizer.runLocalization(nmAddr));
+      int nRet = localizer.runLocalization(nmAddr);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(String.format("nRet: %d", nRet));
+      }
+      System.exit(nRet);
     } catch (Throwable e) {
       // Print error to stdout so that LCE can use it.
       e.printStackTrace(System.out);
+      LOG.error("Exception in main:", e);
       throw e;
     }
   }

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java

@@ -1088,7 +1088,8 @@ public class ResourceLocalizationService extends CompositeService
               ConverterUtils.toString(
                   context.getContainerId().
                   getApplicationAttemptId().getApplicationId()),
-              localizerId, localDirs, logDirs);
+              localizerId,
+              dirsHandler);
         } else {
           throw new IOException("All disks failed. "
               + dirsHandler.getDisksHealthReport(false));

+ 7 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDefaultContainerExecutor.java

@@ -303,6 +303,7 @@ public class TestDefaultContainerExecutor {
   public void testStartLocalizer()
       throws IOException, InterruptedException {
     InetSocketAddress localizationServerAddress;
+    
     final Path firstDir = new Path(BASE_TMP_PATH, "localDir1");
     List<String> localDirs = new ArrayList<String>();
     final Path secondDir = new Path(BASE_TMP_PATH, "localDir2");
@@ -383,9 +384,14 @@ public class TestDefaultContainerExecutor {
     String appSubmitter = "nobody";
     String appId = "APP_ID";
     String locId = "LOC_ID";
+    
+    LocalDirsHandlerService  dirsHandler = mock(LocalDirsHandlerService.class);
+    when(dirsHandler.getLocalDirs()).thenReturn(localDirs);
+    when(dirsHandler.getLogDirs()).thenReturn(logDirs);
+    
     try {
       mockExec.startLocalizer(nmPrivateCTokensPath, localizationServerAddress,
-          appSubmitter, appId, locId, localDirs, logDirs);
+          appSubmitter, appId, locId, dirsHandler);
     } catch (IOException e) {
       Assert.fail("StartLocalizer failed to copy token file " + e);
     } finally {

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutor.java

@@ -266,7 +266,7 @@ public class TestLinuxContainerExecutor {
     exec.setConf(conf);
 
     exec.startLocalizer(nmPrivateContainerTokensPath, nmAddr, appSubmitter,
-      appId, locId, localDirs, logDirs);
+      appId, locId, dirsHandler);
 
     String locId2 = "container_01_02";
     Path nmPrivateContainerTokensPath2 =
@@ -276,7 +276,7 @@ public class TestLinuxContainerExecutor {
               + String.format(ContainerLocalizer.TOKEN_FILE_NAME_FMT, locId2));
     files.create(nmPrivateContainerTokensPath2, EnumSet.of(CREATE, OVERWRITE));
     exec.startLocalizer(nmPrivateContainerTokensPath2, nmAddr, appSubmitter,
-      appId, locId2, localDirs, logDirs);
+      appId, locId2, dirsHandler);
   }
   
   @Test

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutorWithMocks.java

@@ -185,7 +185,7 @@ public class TestLinuxContainerExecutorWithMocks {
     Path nmPrivateCTokensPath= new Path("file:///bin/nmPrivateCTokensPath");
  
     try {
-      mockExec.startLocalizer(nmPrivateCTokensPath, address, "test", "application_0", "12345", dirsHandler.getLocalDirs(), dirsHandler.getLogDirs());
+      mockExec.startLocalizer(nmPrivateCTokensPath, address, "test", "application_0", "12345", dirsHandler);
       List<String> result=readMockParams();
       Assert.assertEquals(result.size(), 17);
       Assert.assertEquals(result.get(0), YarnConfiguration.DEFAULT_NM_NONSECURE_MODE_LOCAL_USER);

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java

@@ -850,7 +850,7 @@ public class TestResourceLocalizationService {
       ArgumentCaptor<Path> tokenPathCaptor = ArgumentCaptor.forClass(Path.class);
       verify(exec).startLocalizer(tokenPathCaptor.capture(),
           isA(InetSocketAddress.class), eq("user0"), eq(appStr), eq(ctnrStr),
-          isA(List.class), isA(List.class));
+          isA(LocalDirsHandlerService.class));
       Path localizationTokenPath = tokenPathCaptor.getValue();
 
       // heartbeat from localizer

+ 63 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/SecureContainer.apt.vm

@@ -83,11 +83,15 @@ min.user.id=1000#Prevent other super-users
 +---+
 
    
-  ** Windows Secure Container Executor
+  ** Windows Secure Container Executor (WSCE)
   
     The Windows environment secure container executor is the <<<WindowsSecureContainerExecutor>>>.
     It uses the Windows S4U infrastructure to launch the container as the 
-    YARN application user.
+    YARN application user. The WSCE requires the presense of the <<<hadoopwinutilsvc>>>  service. This services
+    is hosted by <<<%HADOOP_HOME%\bin\winutils.exe>>> started with the <<<service>>> command line argument. This
+    service offers some privileged operations that require LocalSystem authority so that the NM is not required
+    to run the entire JVM and all the NM code in an elevated context. The NM interacts with the <<<hadoopwintulsvc>>>
+    service by means of Local RPC (LRPC) via calls JNI to the RCP client hosted in <<<hadoop.dll>>>.
     
   *** Configuration
   
@@ -102,17 +106,71 @@ min.user.id=1000#Prevent other super-users
 
 <property>
   <name>yarn.nodemanager.windows-secure-container-executor.group</name>
-  <value>hadoop</value>
+  <value>yarn</value>
 </property>
 +---+
+  *** wsce-site.xml
+  
+      The hadoopwinutilsvc uses <<<%HADOOP_HOME%\etc\hadoop\wsce_site.xml to configure access to the privileged operations.
 
-      The NodeManager must run as a member of the local <<<Administrators>>> group or as 
-      <<<LocalSystem>>>. It is not enough for the NodeManager to simply impersonate such an user.
++---+
+  <property>
+    <name>yarn.nodemanager.windows-secure-container-executor.impersonate.allowed</name>
+    <value>HadoopUsers</value>
+  </property>
+  
+  <property>
+    <name>yarn.nodemanager.windows-secure-container-executor.impersonate.denied</name>
+    <value>HadoopServices,Administrators</value>
+  </property>
+  
+  <property>
+    <name>yarn.nodemanager.windows-secure-container-executor.allowed</name>
+    <value>nodemanager</value>
+  </property>
+
+  <property>
+    <name>yarn.nodemanager.windows-secure-container-executor.local-dirs</name>
+    <value>nm-local-dir, nm-log-dirs</value>
+  </property>
+
+  <property>
+    <name>yarn.nodemanager.windows-secure-container-executor.job-name</name>
+    <value>nodemanager-job-name</value>
+  </property>  
++---+
+      
+      <<<yarn.nodemanager.windows-secure-container-executor.allowed>>> should contain the name of the service account running the 
+      nodemanager. This user will be allowed to access the hadoopwintuilsvc functions.
+      
+      <<<yarn.nodemanager.windows-secure-container-executor.impersonate.allowed>>> should contain the users that are allowed to create
+      containers in the cluster. These users will be allowed to be impersonated by hadoopwinutilsvc.
+      
+      <<<yarn.nodemanager.windows-secure-container-executor.impersonate.denied>>> should contain users that are explictly forbiden from
+      creating containers. hadoopwinutilsvc will refuse to impersonate these users.
+
+      <<<yarn.nodemanager.windows-secure-container-executor.local-dirs>>> should contain the nodemanager local dirs. hadoopwinutilsvc will
+      allow only file operations under these directories. This should contain the same values as <<<${yarn.nodemanager.local-dirs}, ${yarn.nodemanager.log-dirs}>>> 
+      but note that hadoopwinutilsvc XML configuration processing does not do substitutions so the value must be the final value. All paths 
+      must be absolute and no environment variable substitution will be performed. The paths are compared LOCAL_INVARIANT case insensitive string comparison,
+      the file path validated must start with one of the paths listed in local-dirs configuration. Use comma as path separator:<<<,>>>
+
+      <<<yarn.nodemanager.windows-secure-container-executor.job-name>>> should contain an Windows NT job name that all containers should be added to. 
+      This configuration is optional. If not set, the container is not added to a global NodeManager job. Normally this should be set to the job that the NM is assigned to, 
+      so that killing the NM kills also all containers. Hadoopwinutilsvc will not attempt to create this job, the job must exists when the container is launched.
+      If the value is set and the job does not exists, container launch will fail with error 2 <<<The system cannot find the file specified>>>.
+      Note that this global NM job is not related to the container job, which always gets created for each container and is named after the container ID.
+      This setting controls a global job that spans all containers and the parent NM, and as such it requires nested jobs. 
+      Nested jobs are available only post Windows 8 and Windows Server 2012.
       
   *** Useful Links
     
     * {{{http://msdn.microsoft.com/en-us/magazine/cc188757.aspx}Exploring S4U Kerberos Extensions in Windows Server 2003}}
     
+    * {{{http://msdn.microsoft.com/en-us/library/windows/desktop/hh448388(v=vs.85).aspx}Nested Jobs}}
+
     * {{{https://issues.apache.org/jira/browse/YARN-1063}Winutils needs ability to create task as domain user}}
     
     * {{{https://issues.apache.org/jira/browse/YARN-1972}Implement secure Windows Container Executor}}
+
+    * {{{https://issues.apache.org/jira/browse/YARN-2198}Remove the need to run NodeManager as privileged account for Windows Secure Container Executor}}

이 변경점에서 너무 많은 파일들이 변경되어 몇몇 파일들은 표시되지 않았습니다.