You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ji...@apache.org on 2014/10/23 01:26:57 UTC

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c b/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c
index f275111..933c177 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c
+++ b/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);
+}
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.vcxproj
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.vcxproj b/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.vcxproj
index fc0519d..37b7f31 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.vcxproj
+++ b/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>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/src/main/winutils/main.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/main.c b/hadoop-common-project/hadoop-common/src/main/winutils/main.c
index 0f40774..ac73aec 100644
--- a/hadoop-common-project/hadoop-common/src/main/winutils/main.c
+++ b/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");
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1c235a44/hadoop-common-project/hadoop-common/src/main/winutils/service.c
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/winutils/service.c b/hadoop-common-project/hadoop-common/src/main/winutils/service.c
new file mode 100644
index 0000000..a0f8a66
--- /dev/null
+++ b/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");
+}
+
+