You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by ab...@apache.org on 2015/04/10 13:04:02 UTC

ambari git commit: AMBARI-10271 - [WinTP2] Agent Automatic Bootstrap: Adapt the bootstrap.py to upload bootstrap archive to agent node, unzip and run setupAgent script on windows

Repository: ambari
Updated Branches:
  refs/heads/trunk 4b4bcbfb6 -> 6cb623fd8


AMBARI-10271 - [WinTP2] Agent Automatic Bootstrap: Adapt the bootstrap.py to upload bootstrap archive to agent node, unzip and run setupAgent script on windows


Project: http://git-wip-us.apache.org/repos/asf/ambari/repo
Commit: http://git-wip-us.apache.org/repos/asf/ambari/commit/6cb623fd
Tree: http://git-wip-us.apache.org/repos/asf/ambari/tree/6cb623fd
Diff: http://git-wip-us.apache.org/repos/asf/ambari/diff/6cb623fd

Branch: refs/heads/trunk
Commit: 6cb623fd830958f3272cf1081bd9b70820838818
Parents: 4b4bcbf
Author: Artem Baranchuk <ab...@hortonworks.con>
Authored: Fri Apr 10 14:03:39 2015 +0300
Committer: Artem Baranchuk <ab...@hortonworks.con>
Committed: Fri Apr 10 14:03:39 2015 +0300

----------------------------------------------------------------------
 ambari-server/conf/windows/ambari.properties    |   4 +-
 .../src/main/assemblies/server-windows.xml      |  11 +-
 .../ambari/server/bootstrap/BSRunner.java       |  31 +-
 ambari-server/src/main/python/bootstrap.py      | 341 +++++++++++++------
 .../windows/psremoting/Create-RemotingDir.ps1   |  44 +++
 .../windows/psremoting/Run-RemotingScript.ps1   |  44 +++
 .../windows/psremoting/Send-RemotingFile.ps1    | 117 +++++++
 .../windows/psremoting/Unzip-RemotingFile.ps1   |  62 ++++
 ambari-server/src/test/python/TestBootstrap.py  |  55 +--
 9 files changed, 560 insertions(+), 149 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/6cb623fd/ambari-server/conf/windows/ambari.properties
----------------------------------------------------------------------
diff --git a/ambari-server/conf/windows/ambari.properties b/ambari-server/conf/windows/ambari.properties
index d0fb6dd..58c8d9f 100644
--- a/ambari-server/conf/windows/ambari.properties
+++ b/ambari-server/conf/windows/ambari.properties
@@ -37,8 +37,8 @@ common.services.path=resources\\common-services
 server.version.file=version
 webapp.dir=web
 bootstrap.dir=bootstrap
-bootstrap.script=bootstrap\\bootstrap.py
-bootstrap.setup_agent.script=bootstrap\\setupAgent.py
+bootstrap.script=sbin\\bootstrap.py
+bootstrap.setup_agent.script=resources\\bootstrap.zip
 api.authenticate=true
 server.connection.max.idle.millis=900000
 server.fqdn.service.url=http://127.0.0.1/latest/meta-data/public-hostname

http://git-wip-us.apache.org/repos/asf/ambari/blob/6cb623fd/ambari-server/src/main/assemblies/server-windows.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/assemblies/server-windows.xml b/ambari-server/src/main/assemblies/server-windows.xml
index 8b5483a..1fcd191 100644
--- a/ambari-server/src/main/assemblies/server-windows.xml
+++ b/ambari-server/src/main/assemblies/server-windows.xml
@@ -33,7 +33,7 @@
     </file>
     <file>
       <source>${project.build.directory}/bootstrap.zip</source>
-      <outputDirectory>/ambari-server-${project.version}/bootstrap</outputDirectory>
+      <outputDirectory>/ambari-server-${project.version}/resources</outputDirectory>
     </file>
     <file>
       <source>${basedir}/conf/windows/ambari.properties</source>
@@ -61,7 +61,7 @@
     </file>
     <file>
       <source>${basedir}/src/main/python/bootstrap.py</source>
-      <outputDirectory>/ambari-server-${project.version}/bootstrap</outputDirectory>
+      <outputDirectory>/ambari-server-${project.version}/sbin</outputDirectory>
     </file>
     <file>
       <source>${basedir}/src/main/windows/ambari-server.cmd</source>
@@ -84,6 +84,13 @@
   <fileSets>
     <!-- Distro files, readme, licenses, etc -->
     <fileSet>
+      <directory>${basedir}/src/main/windows/psremoting/</directory>
+      <outputDirectory>/ambari-server-${project.version}/sbin</outputDirectory>
+      <includes>
+        <include>*.ps1</include>
+      </includes>
+    </fileSet>
+    <fileSet>
       <directory>${basedir}/../ambari-admin/target</directory>
       <outputDirectory>/ambari-server-${project.version}/resources/views</outputDirectory>
       <includes>

http://git-wip-us.apache.org/repos/asf/ambari/blob/6cb623fd/ambari-server/src/main/java/org/apache/ambari/server/bootstrap/BSRunner.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/bootstrap/BSRunner.java b/ambari-server/src/main/java/org/apache/ambari/server/bootstrap/BSRunner.java
index 331b309..fe873d7 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/bootstrap/BSRunner.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/bootstrap/BSRunner.java
@@ -20,8 +20,7 @@ package org.apache.ambari.server.bootstrap;
 import java.io.File;
 import java.io.IOException;
 import java.io.PrintWriter;
-import java.util.Arrays;
-import java.util.List;
+import java.util.*;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledFuture;
@@ -196,14 +195,36 @@ class BSRunner extends Thread {
       command[9] = this.serverPort+"";
       command[10] = userRunAs;
       command[11] = (this.passwordFile==null) ? "null" : this.passwordFile.toString();
+
+      Map<String, String> envVariables = new HashMap<String, String>();
+
+      if (System.getProperty("os.name").contains("Windows")) {
+        String command2[] = new String[command.length + 1];
+        command2[0] = "python";
+        System.arraycopy(command, 0, command2, 1, command.length);
+        command = command2;
+
+        Map<String, String> envVarsWin = System.getenv();
+        if (envVarsWin != null) {
+          envVariables.putAll(envVarsWin);  //envVarsWin is non-modifiable
+        }
+      }
+
       LOG.info("Host= " + hostString + " bs=" + this.bsScript + " requestDir=" +
           requestIdDir + " user=" + user + " keyfile=" + this.sshKeyFile +
           " passwordFile " + this.passwordFile + " server=" + this.ambariHostname +
-          " version=" + projectVersion + " serverPort=" + this.serverPort + " userRunAs="+ userRunAs);
+          " version=" + projectVersion + " serverPort=" + this.serverPort + " userRunAs=" + userRunAs);
 
-      String[] env = new String[] { "AMBARI_PASSPHRASE=" + agentSetupPassword };
+      envVariables.put("AMBARI_PASSPHRASE", agentSetupPassword);
       if (this.verbose)
-        env = new String[] { env[0], " BS_VERBOSE=\"-vvv\" " };
+        envVariables.put("BS_VERBOSE", "\"-vvv\"");
+
+      String[] env = new String[envVariables.size()];
+      int iVar = 0;
+      for(Map.Entry<String, String> pair : envVariables.entrySet())
+      {
+        env[iVar++] = pair.getKey() + "=" + pair.getValue();
+      }
 
       if (LOG.isDebugEnabled()) {
         LOG.debug(Arrays.toString(command));

http://git-wip-us.apache.org/repos/asf/ambari/blob/6cb623fd/ambari-server/src/main/python/bootstrap.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/python/bootstrap.py b/ambari-server/src/main/python/bootstrap.py
index 692ce27..a3b561d 100755
--- a/ambari-server/src/main/python/bootstrap.py
+++ b/ambari-server/src/main/python/bootstrap.py
@@ -32,8 +32,13 @@ import threading
 import traceback
 import re
 from datetime import datetime
-from resource_management.core.shell import quote_bash_args
-from ambari_commons.os_check import OSCheck
+from ambari_commons import OSCheck, OSConst
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+
+if OSCheck.is_windows_family():
+  from ambari_commons.os_utils import run_os_command, run_in_shell
+else:
+  from resource_management.core.shell import quote_bash_args
 
 AMBARI_PASSPHRASE_VAR_NAME = "AMBARI_PASSPHRASE"
 HOST_BOOTSTRAP_TIMEOUT = 300
@@ -44,6 +49,10 @@ POLL_INTERVAL_SEC = 1
 DEBUG = False
 DEFAULT_AGENT_TEMP_FOLDER = "/var/lib/ambari-agent/data/tmp"
 PYTHON_ENV="env PYTHONPATH=$PYTHONPATH:" + DEFAULT_AGENT_TEMP_FOLDER
+CREATE_REMOTING_DIR_SCRIPT_NAME = "Create-RemotingDir.ps1"
+SEND_REMOTING_FILE_SCRIPT_NAME = "Send-RemotingFile.ps1"
+UNZIP_REMOTING_SCRIPT_NAME = "Unzip-RemotingFile.ps1"
+RUN_REMOTING_SCRIPT_NAME = "Run-RemotingScript.ps1"
 
 
 class HostLog:
@@ -103,7 +112,6 @@ class SCP:
     return {"exitstatus": scpstat.returncode, "log": log, "errormsg": errorMsg}
 
 
-
 class SSH:
   """ Ssh implementation of this """
   def __init__(self, user, sshkey_file, host, command, bootdir, host_log, errorMessage = None):
@@ -143,15 +151,45 @@ class SSH:
     return  {"exitstatus": sshstat.returncode, "log": log, "errormsg": errorMsg}
 
 
+class PSR:
+  """ PowerShell Remoting implementation of this """
+  def __init__(self, command, host, host_log, params=None, errorMessage=None):
+    self.command = command
+    self.host = host
+    self.host_log = host_log
+    self.params = params
+    self.errorMessage = errorMessage
+    pass
+
+  def run(self):
+    #os.environ['COMSPEC'] = 'c:\\System32\\WindowsPowerShell\\v1.0\\powershell.exe'
+    psrcommand = ["powershell.exe",
+                  "-NoProfile",
+                  "-InputFormat", "Text",
+                  "-ExecutionPolicy", "unrestricted",
+                  "-Command", self.command]
+    if self.params:
+      psrcommand.extend([self.params])
+    if DEBUG:
+      self.host_log.write("Running PowerShell command " + ' '.join(psrcommand))
+    self.host_log.write("==========================")
+    self.host_log.write("\nCommand start time " + datetime.now().strftime('%Y-%m-%d %H:%M:%S'))
+    retcode, stdout, stderr = run_os_command(psrcommand)
+    errorMsg = stderr
+    if self.errorMessage and retcode != 0:
+      errorMsg = self.errorMessage + "\n" + stderr
+    log = stdout + "\n" + errorMsg
+    self.host_log.write(log)
+    self.host_log.write("PowerShell command execution finished")
+    self.host_log.write("host=" + self.host + ", exitcode=" + str(retcode))
+    self.host_log.write("Command end time " + datetime.now().strftime('%Y-%m-%d %H:%M:%S'))
+    return {"exitstatus": retcode, "log": log, "errormsg": errorMsg}
+
 
 class Bootstrap(threading.Thread):
   """ Bootstrap the agent on a separate host"""
-  TEMP_FOLDER = DEFAULT_AGENT_TEMP_FOLDER
-  OS_CHECK_SCRIPT_FILENAME = "os_check_type.py"
-  AMBARI_REPO_FILENAME = "ambari"
   SETUP_SCRIPT_FILENAME = "setupAgent.py"
-  PASSWORD_FILENAME = "host_pass"
-  ambari_commons="/usr/lib/python2.6/site-packages/ambari_commons"
+  AMBARI_REPO_FILENAME = "ambari"
 
   def __init__(self, host, shared_state):
     threading.Thread.__init__(self)
@@ -170,6 +208,152 @@ class Bootstrap(threading.Thread):
     else:
       self.AMBARI_REPO_FILENAME = self.AMBARI_REPO_FILENAME + ".repo"
 
+  # This method is needed  to implement the descriptor protocol (make object
+  # to pass self reference to mockups)
+  def __get__(self, obj, objtype):
+    def _call(*args, **kwargs):
+      self(obj, *args, **kwargs)
+    return _call
+
+  def try_to_execute(self, action):
+    last_retcode = {"exitstatus": 177, "log":"Try to execute '{0}'".format(str(action)), "errormsg":"Execute of '{0}' failed".format(str(action))}
+    try:
+      retcode = action()
+      if isinstance(retcode, int):
+        last_retcode["exitstatus"] = retcode
+      else:
+        last_retcode = retcode
+    except Exception:
+      self.host_log.write("Traceback: " + traceback.format_exc())
+    return last_retcode
+
+  def getAmbariVersion(self):
+    ambari_version = self.shared_state.ambari_version
+    if ambari_version is None or ambari_version == "null":
+      return ""
+    else:
+      return ambari_version
+
+  def createDoneFile(self, retcode):
+    """ Creates .done file for current host. These files are later read from Java code.
+    If .done file for any host is not created, the bootstrap will hang or fail due to timeout"""
+    params = self.shared_state
+    doneFilePath = os.path.join(params.bootdir, self.host + ".done")
+    if not os.path.exists(doneFilePath):
+      doneFile = open(doneFilePath, "w+")
+      doneFile.write(str(retcode))
+      doneFile.close()
+
+  def getStatus(self):
+    return self.status
+
+  def interruptBootstrap(self):
+    """
+    Thread is not really interrupted (moreover, Python seems to have no any
+    stable/portable/official api to do that: _Thread__stop only marks thread
+    as stopped). The bootstrap thread is marked as a daemon at init, and will
+    exit when the main parallel bootstrap thread exits.
+    All we need to do now is a proper logging and creating .done file
+    """
+    self.host_log.write("Automatic Agent registration timed out (timeout = {0} seconds). " \
+                        "Check your network connectivity and retry registration," \
+                        " or use manual agent registration.".format(HOST_BOOTSTRAP_TIMEOUT))
+    self.createDoneFile(199)
+
+
+
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class BootstrapWindows(Bootstrap):
+  bootstrapArchiveName = "bootstrap.zip"
+
+  def getTempFolder(self):
+    installationDrive = os.path.splitdrive(self.shared_state.script_dir)[0]
+    return os.path.join(installationDrive, os.sep, "var", "temp", "bootstrap", self.getAmbariVersion())
+
+  def createTargetDir(self):
+    # Creating target dir
+    self.host_log.write("==========================\n")
+    self.host_log.write("Creating target directory...")
+    command = os.path.join(self.shared_state.script_dir, CREATE_REMOTING_DIR_SCRIPT_NAME)
+    psr = PSR(command, self.host, self.host_log, params="{0} {1}".format(self.host, self.getTempFolder()))
+    retcode = psr.run()
+    self.host_log.write("\n")
+    return retcode
+
+  def unzippingBootstrapArchive(self):
+    # Unzipping bootstrap archive
+    zipFile = os.path.join(self.getTempFolder(), self.bootstrapArchiveName)
+    self.host_log.write("==========================\n")
+    self.host_log.write("Unzipping bootstrap archive...")
+    command = os.path.join(self.shared_state.script_dir, UNZIP_REMOTING_SCRIPT_NAME)
+    psr = PSR(command, self.host, self.host_log, params="{0} {1} {2}".format(self.host, zipFile, self.getTempFolder()))
+    result = psr.run()
+    self.host_log.write("\n")
+    return result
+
+  def copyBootstrapArchive(self):
+    # Copying the bootstrap archive file
+    fileToCopy = os.path.join(self.shared_state.script_dir, os.path.dirname(self.shared_state.script_dir), self.shared_state.setup_agent_file)
+    target = os.path.join(self.getTempFolder(), self.bootstrapArchiveName)
+    self.host_log.write("==========================\n")
+    self.host_log.write("Copying bootstrap archive...")
+    command = os.path.join(self.shared_state.script_dir, SEND_REMOTING_FILE_SCRIPT_NAME)
+    psr = PSR(command, self.host, self.host_log, params="{0} {1} {2}".format(self.host, fileToCopy, target))
+    result = psr.run()
+    self.host_log.write("\n")
+    return result
+
+  def getRunSetupCommand(self, expected_hostname):
+    setupFile = os.path.join(self.getTempFolder(), self.SETUP_SCRIPT_FILENAME)
+    msi_url = 'http://{0}:{1}/resources/ambari-agent.msi'.format(self.shared_state.ambari_server, self.shared_state.server_port)
+    server = self.shared_state.ambari_server
+    version = self.getAmbariVersion()
+    return ' '.join(['python', setupFile, msi_url, server, version])
+
+  def runSetupAgent(self):
+    self.host_log.write("==========================\n")
+    self.host_log.write("Running setup agent script...")
+    command = os.path.join(self.shared_state.script_dir, RUN_REMOTING_SCRIPT_NAME)
+    psr = PSR(command, self.host, self.host_log, params="{0} \"{1}\"".format(self.host, self.getRunSetupCommand(self.host)))
+    retcode = psr.run()
+    self.host_log.write("\n")
+    return retcode
+
+  def run(self):
+    """ Copy files and run commands on remote host """
+    self.status["start_time"] = time.time()
+    # Population of action queue
+    action_queue = [self.createTargetDir,
+                    self.copyBootstrapArchive,
+                    self.unzippingBootstrapArchive,
+                    self.runSetupAgent
+    ]
+    # Execution of action queue
+    last_retcode = 0
+    while action_queue and last_retcode == 0:
+      action = action_queue.pop(0)
+      ret = self.try_to_execute(action)
+      last_retcode = ret["exitstatus"]
+      err_msg = ret["errormsg"]
+      std_out = ret["log"]
+    # Checking execution result
+    if last_retcode != 0:
+      message = "ERROR: Bootstrap of host {0} fails because previous action " \
+                "finished with non-zero exit code ({1})\nERROR MESSAGE: {2}\nSTDOUT: {3}".format(self.host, last_retcode, err_msg, std_out)
+      self.host_log.write(message)
+      logging.error(message)
+
+    self.createDoneFile(last_retcode)
+    self.status["return_code"] = last_retcode
+
+
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class BootstrapDefault(Bootstrap):
+  ambari_commons="/usr/lib/python2.6/site-packages/ambari_commons"
+  TEMP_FOLDER = DEFAULT_AGENT_TEMP_FOLDER
+  OS_CHECK_SCRIPT_FILENAME = "os_check_type.py"
+  PASSWORD_FILENAME = "host_pass"
 
   def getRemoteName(self, filename):
     full_name = os.path.join(self.TEMP_FOLDER, filename)
@@ -178,21 +362,12 @@ class Bootstrap(threading.Thread):
       remote_files[full_name] = self.generateRandomFileName(full_name)
     return remote_files[full_name]
 
-
   def generateRandomFileName(self, filename):
     if filename is None:
       return self.getUtime()
     else:
       name, ext = os.path.splitext(filename)
       return str(name) + str(self.getUtime()) + str(ext)
-
-
-  # This method is needed  to implement the descriptor protocol (make object
-  # to pass self reference to mockups)
-  def __get__(self, obj, objtype):
-    def _call(*args, **kwargs):
-      self(obj, *args, **kwargs)
-    return _call
   
   def getRepoDir(self):
     if OSCheck.is_redhat_family():
@@ -204,7 +379,6 @@ class Bootstrap(threading.Thread):
     else:
       raise Exception("Unsupported OS family '{0}'".format(OSCheck.get_os_family()))
 
-
   def getRepoFile(self):
     """ Ambari repo file for Ambari."""
     return os.path.join(self.getRepoDir(), self.AMBARI_REPO_FILENAME)
@@ -228,23 +402,6 @@ class Bootstrap(threading.Thread):
     password_file = self.shared_state.password_file
     return password_file is not None and password_file != 'null'
 
-
-  def createTargetDir(self):
-    # Creating target dir
-    self.host_log.write("==========================\n")
-    self.host_log.write("Creating target directory...")
-    params = self.shared_state
-    user = params.user
-
-    command = "sudo mkdir -p {0} ; sudo chown -R {1} {0}".format(self.TEMP_FOLDER,quote_bash_args(params.user))
-
-    ssh = SSH(params.user, params.sshkey_file, self.host, command,
-              params.bootdir, self.host_log)
-    retcode = ssh.run()
-    self.host_log.write("\n")
-    return retcode
-
-
   def copyOsCheckScript(self):
     # Copying the os check script file
     fileToCopy = self.getOsCheckScript()
@@ -271,13 +428,11 @@ class Bootstrap(threading.Thread):
     self.host_log.write("\n")
     return result
 
-
   def getMoveRepoFileWithPasswordCommand(self, targetDir):
     return "sudo -S mv " + str(self.getRemoteName(self.AMBARI_REPO_FILENAME)) \
            + " " + os.path.join(str(targetDir), self.AMBARI_REPO_FILENAME) + \
            " < " + str(self.getPasswordFile())
 
-
   def getMoveRepoFileWithoutPasswordCommand(self, targetDir):
     return "sudo mv " + str(self.getRemoteName(self.AMBARI_REPO_FILENAME)) \
            + " " + os.path.join(str(targetDir), self.AMBARI_REPO_FILENAME)
@@ -336,14 +491,6 @@ class Bootstrap(threading.Thread):
 
     return max(retcode1["exitstatus"], retcode2["exitstatus"], retcode3["exitstatus"])
 
-
-  def getAmbariVersion(self):
-    ambari_version = self.shared_state.ambari_version
-    if ambari_version is None or ambari_version == "null":
-      return ""
-    else:
-      return ambari_version
-
   def getAmbariPort(self):
     server_port = self.shared_state.server_port
     if server_port is None or server_port == "null":
@@ -363,7 +510,6 @@ class Bootstrap(threading.Thread):
            " " + str(passphrase) + " " + str(server)+ " " + quote_bash_args(str(user_run_as)) + " " + str(version) + \
            " " + str(port) + " < " + str(passwordFile)
 
-
   def getRunSetupWithoutPasswordCommand(self, expected_hostname):
     setupFile=self.getRemoteName(self.SETUP_SCRIPT_FILENAME)
     passphrase=os.environ[AMBARI_PASSPHRASE_VAR_NAME]
@@ -375,14 +521,6 @@ class Bootstrap(threading.Thread):
            " " + str(passphrase) + " " + str(server)+ " " + quote_bash_args(str(user_run_as)) + " " + str(version) + \
            " " + str(port)
 
-
-  def getRunSetupCommand(self, expected_hostname):
-    if self.hasPassword():
-      return self.getRunSetupWithPasswordCommand(expected_hostname)
-    else:
-      return self.getRunSetupWithoutPasswordCommand(expected_hostname)
-
-
   def runOsCheckScript(self):
     params = self.shared_state
     self.host_log.write("==========================\n")
@@ -398,29 +536,6 @@ class Bootstrap(threading.Thread):
     self.host_log.write("\n")
     return retcode
 
-
-  def runSetupAgent(self):
-    params = self.shared_state
-    self.host_log.write("==========================\n")
-    self.host_log.write("Running setup agent script...")
-    command = self.getRunSetupCommand(self.host)
-    ssh = SSH(params.user, params.sshkey_file, self.host, command,
-              params.bootdir, self.host_log)
-    retcode = ssh.run()
-    self.host_log.write("\n")
-    return retcode
-
-
-  def createDoneFile(self, retcode):
-    """ Creates .done file for current host. These files are later read from Java code.
-    If .done file for any host is not created, the bootstrap will hang or fail due to timeout"""
-    params = self.shared_state
-    doneFilePath = os.path.join(params.bootdir, self.host + ".done")
-    if not os.path.exists(doneFilePath):
-      doneFile = open(doneFilePath, "w+")
-      doneFile.write(str(retcode))
-      doneFile.close()
-
   def checkSudoPackage(self):
     """ Checking 'sudo' package on remote host """
     self.host_log.write("==========================\n")
@@ -458,7 +573,6 @@ class Bootstrap(threading.Thread):
     self.host_log.write("Copying password file finished")
     return max(retcode1["exitstatus"], retcode2["exitstatus"])
 
-
   def changePasswordFileModeOnHost(self):
     # Change password file mode to 600
     self.host_log.write("Changing password file mode...")
@@ -470,7 +584,6 @@ class Bootstrap(threading.Thread):
     self.host_log.write("Change password file mode on host finished")
     return retcode
 
-
   def deletePasswordFile(self):
     # Deleting the password file
     self.host_log.write("Deleting password file...")
@@ -482,17 +595,37 @@ class Bootstrap(threading.Thread):
     self.host_log.write("Deleting password file finished")
     return retcode
 
-  def try_to_execute(self, action):
-    last_retcode = {"exitstatus": 177, "log":"Try to execute '{0}'".format(str(action)), "errormsg":"Execute of '{0}' failed".format(str(action))}
-    try:
-      retcode = action()
-      if isinstance(retcode, int):
-        last_retcode["exitstatus"] = retcode
-      else:
-        last_retcode = retcode
-    except Exception:
-      self.host_log.write("Traceback: " + traceback.format_exc())
-    return last_retcode
+  def createTargetDir(self):
+    # Creating target dir
+    self.host_log.write("==========================\n")
+    self.host_log.write("Creating target directory...")
+    params = self.shared_state
+    user = params.user
+
+    command = "sudo mkdir -p {0} ; sudo chown -R {1} {0}".format(self.TEMP_FOLDER,quote_bash_args(params.user))
+
+    ssh = SSH(params.user, params.sshkey_file, self.host, command,
+              params.bootdir, self.host_log)
+    retcode = ssh.run()
+    self.host_log.write("\n")
+    return retcode
+
+  def getRunSetupCommand(self, expected_hostname):
+    if self.hasPassword():
+      return self.getRunSetupWithPasswordCommand(expected_hostname)
+    else:
+      return self.getRunSetupWithoutPasswordCommand(expected_hostname)
+
+  def runSetupAgent(self):
+    params = self.shared_state
+    self.host_log.write("==========================\n")
+    self.host_log.write("Running setup agent script...")
+    command = self.getRunSetupCommand(self.host)
+    ssh = SSH(params.user, params.sshkey_file, self.host, command,
+              params.bootdir, self.host_log)
+    retcode = ssh.run()
+    self.host_log.write("\n")
+    return retcode
 
   def run(self):
     """ Copy files and run commands on remote host """
@@ -540,24 +673,6 @@ class Bootstrap(threading.Thread):
 
 
 
-  def getStatus(self):
-    return self.status
-
-  def interruptBootstrap(self):
-    """
-    Thread is not really interrupted (moreover, Python seems to have no any
-    stable/portable/official api to do that: _Thread__stop only marks thread
-    as stopped). The bootstrap thread is marked as a daemon at init, and will
-    exit when the main parallel bootstrap thread exits.
-    All we need to do now is a proper logging and creating .done file
-    """
-    self.host_log.write("Automatic Agent registration timed out (timeout = {0} seconds). " \
-                        "Check your network connectivity and retry registration," \
-                        " or use manual agent registration.".format(HOST_BOOTSTRAP_TIMEOUT))
-    self.createDoneFile(199)
-
-
-
 class PBootstrap:
   """ BootStrapping the agents on a list of hosts"""
   def __init__(self, hosts, sharedState):
@@ -604,7 +719,6 @@ class PBootstrap:
     logging.info("Finished parallel bootstrap")
 
 
-
 class SharedState:
   def __init__(self, user, sshkey_file, script_dir, boottmpdir, setup_agent_file,
                ambari_server, cluster_os_type, ambari_version, server_port,
@@ -651,11 +765,12 @@ def main(argv=None):
   user_run_as = onlyargs[9]
   passwordFile = onlyargs[10]
 
-  # ssh doesn't like open files
-  subprocess.Popen(["chmod", "600", sshkey_file], stdout=subprocess.PIPE)
+  if not OSCheck.is_windows_family():
+    # ssh doesn't like open files
+    subprocess.Popen(["chmod", "600", sshkey_file], stdout=subprocess.PIPE)
 
-  if passwordFile is not None and passwordFile != 'null':
-    subprocess.Popen(["chmod", "600", passwordFile], stdout=subprocess.PIPE)
+    if passwordFile is not None and passwordFile != 'null':
+      subprocess.Popen(["chmod", "600", passwordFile], stdout=subprocess.PIPE)
 
   logging.info("BootStrapping hosts " + pprint.pformat(hostList) +
                " using " + scriptDir + " cluster primary OS: " + cluster_os_type +

http://git-wip-us.apache.org/repos/asf/ambari/blob/6cb623fd/ambari-server/src/main/windows/psremoting/Create-RemotingDir.ps1
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/windows/psremoting/Create-RemotingDir.ps1 b/ambari-server/src/main/windows/psremoting/Create-RemotingDir.ps1
new file mode 100644
index 0000000..638dc56
--- /dev/null
+++ b/ambari-server/src/main/windows/psremoting/Create-RemotingDir.ps1
@@ -0,0 +1,44 @@
+# 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
+
+[CmdletBinding()]
+param (
+    [Parameter(Mandatory=$true)]
+    [string]
+    $ComputerName,
+
+    [Parameter(Mandatory=$true)]
+    [string]
+    $Path
+)
+
+function RunRemote {
+    $ret = Invoke-Command -ComputerName $ComputerName -ScriptBlock {
+        try {
+            Write-Host "Creating SetupSoftwareAndFiles Folder"
+            $out = New-Item -Path $args[0] -type directory -Force
+            Write-Host "Folder creation complete"
+        }
+        catch {
+            Write-Host $_.Exception.Message
+            Write-Host $_.Exception.ItemName
+            return -1
+        }
+        return 0
+    } -ArgumentList $Path
+    return $ret
+}
+$status = Invoke-Expression "RunRemote"
+[environment]::exit($status)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/6cb623fd/ambari-server/src/main/windows/psremoting/Run-RemotingScript.ps1
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/windows/psremoting/Run-RemotingScript.ps1 b/ambari-server/src/main/windows/psremoting/Run-RemotingScript.ps1
new file mode 100644
index 0000000..a014e4a
--- /dev/null
+++ b/ambari-server/src/main/windows/psremoting/Run-RemotingScript.ps1
@@ -0,0 +1,44 @@
+# 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
+
+[CmdletBinding()]
+param (
+    [Parameter(Mandatory=$true)]
+    [string]
+    $ComputerName,
+
+    [Parameter(Mandatory=$true)]
+    [string]
+    $Command
+)
+
+function RunRemote {
+    $ret = Invoke-Command -ComputerName $ComputerName -ScriptBlock {
+        $command = $args[0]
+        try {
+            $out = Invoke-Expression -Command $command
+            Write-Host $out
+            return $LastExitCode
+        }
+        catch {
+            Write-Host $_.Exception.Message
+            Write-Host $_.Exception.ItemName
+            return -1
+        }
+    } -ArgumentList $Command
+    return $ret
+}
+$status = Invoke-Expression "RunRemote"
+[environment]::exit($status)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/6cb623fd/ambari-server/src/main/windows/psremoting/Send-RemotingFile.ps1
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/windows/psremoting/Send-RemotingFile.ps1 b/ambari-server/src/main/windows/psremoting/Send-RemotingFile.ps1
new file mode 100644
index 0000000..76c79df
--- /dev/null
+++ b/ambari-server/src/main/windows/psremoting/Send-RemotingFile.ps1
@@ -0,0 +1,117 @@
+# 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
+
+[CmdletBinding()]
+param (
+    [Parameter(Mandatory=$true)]
+    [string]
+    $ComputerName,
+
+    [Parameter(Mandatory=$true)]
+    [string]
+    $Path,
+
+    [Parameter(Mandatory=$true)]
+    [string]
+    $Destination,
+
+    [int]
+    $TransferChunkSize = 0x10000
+)
+
+function Initialize-TempScript ($Path) {
+    "<# DATA" | Set-Content -Path $Path
+}
+
+function Complete-Chunk () {
+@"
+DATA #>
+`$TransferPath = `$Env:TEMP | Join-Path -ChildPath '$TransferId'
+`$InData = `$false
+`$WriteStream = [IO.File]::OpenWrite(`$TransferPath)
+try {
+    `$WriteStream.Seek(0, 'End') | Out-Null
+    `$MyInvocation.MyCommand.Definition -split "``n" | ForEach-Object {
+        if (`$InData) {
+            `$InData = -not `$_.StartsWith('DATA #>')
+            if (`$InData) {
+                `$WriteBuffer = [Convert]::FromBase64String(`$_)
+                `$WriteStream.Write(`$WriteBuffer, 0, `$WriteBuffer.Length)
+            }
+        } else {
+            `$InData = `$_.StartsWith('<# DATA')
+        }
+    }
+} finally {
+    `$WriteStream.Close()
+}
+"@
+}
+
+function Complete-FinalChunk ($Destination) {
+@"
+`$TransferPath | Move-Item -Destination '$Destination' -Force
+"@
+}
+
+$ErrorActionPreference = 'Stop'
+Set-StrictMode -Version Latest
+
+$EncodingChunkSize = 57 * 100
+if ($EncodingChunkSize % 57 -ne 0) {
+    throw "EncodingChunkSize must be a multiple of 57"
+}
+
+$TransferId = [Guid]::NewGuid().ToString()
+
+
+$Path = ($Path | Resolve-Path).ProviderPath
+$ReadBuffer = New-Object -TypeName byte[] -ArgumentList $EncodingChunkSize
+
+$TempPath = ([IO.Path]::GetTempFileName() | % { $_ | Move-Item -Destination "$_.ps1" -PassThru}).FullName
+$Session = New-PSSession -ComputerName $ComputerName
+$ReadStream = [IO.File]::OpenRead($Path)
+
+$ChunkCount = 0
+Initialize-TempScript -Path $TempPath
+
+try {
+    do {
+        $ReadCount = $ReadStream.Read($ReadBuffer, 0, $EncodingChunkSize)
+        if ($ReadCount -gt 0) {
+            [Convert]::ToBase64String($ReadBuffer, 0, $ReadCount, 'InsertLineBreaks') |
+                Add-Content -Path $TempPath
+        }
+        $ChunkCount += $ReadCount
+        if ($ChunkCount -ge $TransferChunkSize -or $ReadCount -eq 0) {
+            # send
+            Complete-Chunk | Add-Content -Path $TempPath
+            if ($ReadCount -eq 0) {
+                Complete-FinalChunk -Destination $Destination | Add-Content -Path $TempPath
+                Write-Verbose "Sending final chunk"
+            }
+            Invoke-Command -Session $Session -FilePath $TempPath
+
+            # reset
+            $ChunkCount = 0
+            Initialize-TempScript -Path $TempPath
+        }
+    } while ($ReadCount -gt 0)
+} finally {
+    if ($ReadStream) { $ReadStream.Close() }
+    $Session | Remove-PSSession
+    $TempPath | Remove-Item
+}
+

http://git-wip-us.apache.org/repos/asf/ambari/blob/6cb623fd/ambari-server/src/main/windows/psremoting/Unzip-RemotingFile.ps1
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/windows/psremoting/Unzip-RemotingFile.ps1 b/ambari-server/src/main/windows/psremoting/Unzip-RemotingFile.ps1
new file mode 100644
index 0000000..49ecb00
--- /dev/null
+++ b/ambari-server/src/main/windows/psremoting/Unzip-RemotingFile.ps1
@@ -0,0 +1,62 @@
+# 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
+
+[CmdletBinding()]
+param (
+    [Parameter(Mandatory=$true)]
+    [string]
+    $ComputerName,
+
+  [Parameter(Mandatory=$true)]
+    [string]
+    $Path,
+
+  [Parameter(Mandatory=$true)]
+    [string]
+    $Destination,
+
+  [int]
+    $Overwrite = 0x14
+)
+
+function RunRemote {
+  $ret = Invoke-Command -ComputerName $ComputerName -ScriptBlock {
+    $fileName = $args[0]
+    $destination = $args[1]
+    try {
+      if (!(Test-Path $fileName)) {
+        Write-Host "Archive $fileName does not exist"
+        return -1
+      }
+      if (!(Test-Path $destination)) {
+        Write-Host "Destination $destination does not exist. Creating.."
+        New-Item -Path $destination -type directory -Force
+        Write-Host "Destination path $destination created!"
+      }
+      $shell_app = new-object -com shell.application
+      $zip_file = $shell_app.namespace($fileName)
+      $shell_app.namespace($destination).Copyhere($zip_file.items(), $args[2])
+    }
+    catch {
+      Write-Host $_.Exception.Message
+      Write-Host $_.Exception.ItemName
+      return -1
+    }
+    return 0
+  } -ArgumentList $Path, $Destination, $Overwrite
+  return $ret
+}
+$status = Invoke-Expression "RunRemote"
+[environment]::exit($status)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/6cb623fd/ambari-server/src/test/python/TestBootstrap.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/TestBootstrap.py b/ambari-server/src/test/python/TestBootstrap.py
index 0dadaed..8066d13 100644
--- a/ambari-server/src/test/python/TestBootstrap.py
+++ b/ambari-server/src/test/python/TestBootstrap.py
@@ -26,15 +26,16 @@ import tempfile
 import pprint
 
 from ambari_commons.os_check import OSCheck
-from bootstrap import PBootstrap, Bootstrap, SharedState, HostLog, SCP, SSH
+from bootstrap import PBootstrap, Bootstrap, BootstrapDefault, SharedState, HostLog, SCP, SSH
 from unittest import TestCase
 from subprocess import Popen
 from bootstrap import AMBARI_PASSPHRASE_VAR_NAME
 from mock.mock import MagicMock, call
 from mock.mock import patch
 from mock.mock import create_autospec
+from only_for_platform import only_for_platform, PLATFORM_LINUX
 
-
+@only_for_platform(PLATFORM_LINUX)
 class TestBootstrap(TestCase):
 
   def setUp(self):
@@ -313,7 +314,7 @@ class TestBootstrap(TestCase):
     self.assertEquals(ocs, "scriptDir/os_check_type.py")
 
 
-  @patch.object(Bootstrap, "getRemoteName")
+  @patch.object(BootstrapDefault, "getRemoteName")
   def test_getOsCheckScriptRemoteLocation(self, getRemoteName_mock):
     shared_state = SharedState("root", "sshkey_file", "scriptDir", "bootdir",
                                "setupAgentFile", "ambariServer", "centos6",
@@ -325,7 +326,7 @@ class TestBootstrap(TestCase):
     self.assertEquals(ocs, v)
 
 
-  @patch.object(Bootstrap, "is_suse")
+  @patch.object(BootstrapDefault, "is_suse")
   def test_getRepoFile(self, is_suse_mock):
     shared_state = SharedState("root", "sshkey_file", "scriptDir", "bootdir",
                                "setupAgentFile", "ambariServer", "centos6",
@@ -355,8 +356,8 @@ class TestBootstrap(TestCase):
                      "sudo mkdir -p /var/lib/ambari-agent/data/tmp ; "
                      "sudo chown -R root /var/lib/ambari-agent/data/tmp")
 
-  @patch.object(Bootstrap, "getOsCheckScript")
-  @patch.object(Bootstrap, "getOsCheckScriptRemoteLocation")
+  @patch.object(BootstrapDefault, "getOsCheckScript")
+  @patch.object(BootstrapDefault, "getOsCheckScriptRemoteLocation")
   @patch.object(SCP, "__init__")
   @patch.object(SCP, "run")
   @patch.object(HostLog, "write")
@@ -379,8 +380,8 @@ class TestBootstrap(TestCase):
     self.assertEqual(remote_file, "OsCheckScriptRemoteLocation")
 
 
-  @patch.object(Bootstrap, "getRemoteName")
-  @patch.object(Bootstrap, "hasPassword")
+  @patch.object(BootstrapDefault, "getRemoteName")
+  @patch.object(BootstrapDefault, "hasPassword")
   @patch.object(OSCheck, "is_suse_family")
   @patch.object(OSCheck, "is_ubuntu_family")
   @patch.object(OSCheck, "is_redhat_family")
@@ -407,10 +408,10 @@ class TestBootstrap(TestCase):
   @patch.object(OSCheck, "is_suse_family")
   @patch.object(OSCheck, "is_ubuntu_family")
   @patch.object(OSCheck, "is_redhat_family")
-  @patch.object(Bootstrap, "getMoveRepoFileCommand")
-  @patch.object(Bootstrap, "getRepoDir")
-  @patch.object(Bootstrap, "getRepoFile")
-  @patch.object(Bootstrap, "getRemoteName")
+  @patch.object(BootstrapDefault, "getMoveRepoFileCommand")
+  @patch.object(BootstrapDefault, "getRepoDir")
+  @patch.object(BootstrapDefault, "getRepoFile")
+  @patch.object(BootstrapDefault, "getRemoteName")
   @patch.object(SCP, "__init__")
   @patch.object(SCP, "run")
   @patch.object(SSH, "__init__")
@@ -465,7 +466,7 @@ class TestBootstrap(TestCase):
     self.assertEquals(res, expected3["exitstatus"])
 
 
-  @patch.object(Bootstrap, "getOsCheckScriptRemoteLocation")
+  @patch.object(BootstrapDefault, "getOsCheckScriptRemoteLocation")
   @patch.object(SSH, "__init__")
   @patch.object(SSH, "run")
   @patch.object(HostLog, "write")
@@ -488,7 +489,7 @@ class TestBootstrap(TestCase):
 
 
   @patch.object(SSH, "__init__")
-  @patch.object(Bootstrap, "getRunSetupCommand")
+  @patch.object(BootstrapDefault, "getRunSetupCommand")
   @patch.object(SSH, "run")
   @patch.object(HostLog, "write")
   def test_runSetupAgent(self, write_mock, run_mock,
@@ -507,9 +508,9 @@ class TestBootstrap(TestCase):
     self.assertEqual(command, "RunSetupCommand")
 
 
-  @patch.object(Bootstrap, "hasPassword")
-  @patch.object(Bootstrap, "getRunSetupWithPasswordCommand")
-  @patch.object(Bootstrap, "getRunSetupWithoutPasswordCommand")
+  @patch.object(BootstrapDefault, "hasPassword")
+  @patch.object(BootstrapDefault, "getRunSetupWithPasswordCommand")
+  @patch.object(BootstrapDefault, "getRunSetupWithoutPasswordCommand")
   def test_getRunSetupCommand(self, getRunSetupWithoutPasswordCommand_mock,
                               getRunSetupWithPasswordCommand_mock,
                               hasPassword_mock):
@@ -593,7 +594,7 @@ class TestBootstrap(TestCase):
   @patch.object(SSH, "__init__")
   @patch.object(SSH, "run")
   @patch.object(HostLog, "write")
-  @patch.object(Bootstrap, "getPasswordFile")
+  @patch.object(BootstrapDefault, "getPasswordFile")
   def test_deletePasswordFile(self, getPasswordFile_mock, write_mock, run_mock,
                               init_mock):
     shared_state = SharedState("root", "sshkey_file", "scriptDir", "bootdir",
@@ -610,7 +611,7 @@ class TestBootstrap(TestCase):
     self.assertEqual(command, "rm PasswordFile")
 
 
-  @patch.object(Bootstrap, "getPasswordFile")
+  @patch.object(BootstrapDefault, "getPasswordFile")
   @patch.object(SCP, "__init__")
   @patch.object(SCP, "run")
   @patch.object(SSH, "__init__")
@@ -649,7 +650,7 @@ class TestBootstrap(TestCase):
   @patch.object(SSH, "__init__")
   @patch.object(SSH, "run")
   @patch.object(HostLog, "write")
-  @patch.object(Bootstrap, "getPasswordFile")
+  @patch.object(BootstrapDefault, "getPasswordFile")
   def test_changePasswordFileModeOnHost(self, getPasswordFile_mock, write_mock,
                                         run_mock, init_mock):
     shared_state = SharedState("root", "sshkey_file", "scriptDir", "bootdir",
@@ -694,9 +695,9 @@ class TestBootstrap(TestCase):
     self.assertTrue(write_mock.called)
 
 
-  @patch.object(Bootstrap, "try_to_execute")
-  @patch.object(Bootstrap, "hasPassword")
-  @patch.object(Bootstrap, "createDoneFile")
+  @patch.object(BootstrapDefault, "try_to_execute")
+  @patch.object(BootstrapDefault, "hasPassword")
+  @patch.object(BootstrapDefault, "createDoneFile")
   @patch.object(HostLog, "write")
   @patch("logging.warn")
   @patch("logging.error")
@@ -771,7 +772,7 @@ class TestBootstrap(TestCase):
     self.assertEqual(bootstrap_obj.getStatus()["return_code"], 17)
 
 
-  @patch.object(Bootstrap, "createDoneFile")
+  @patch.object(BootstrapDefault, "createDoneFile")
   @patch.object(HostLog, "write")
   def test_interruptBootstrap(self, write_mock, createDoneFile_mock):
     shared_state = SharedState("root", "sshkey_file", "scriptDir", "bootdir",
@@ -786,9 +787,9 @@ class TestBootstrap(TestCase):
   @patch("time.time")
   @patch("logging.warn")
   @patch("logging.info")
-  @patch.object(Bootstrap, "start")
-  @patch.object(Bootstrap, "interruptBootstrap")
-  @patch.object(Bootstrap, "getStatus")
+  @patch.object(BootstrapDefault, "start")
+  @patch.object(BootstrapDefault, "interruptBootstrap")
+  @patch.object(BootstrapDefault, "getStatus")
   def test_PBootstrap(self, getStatus_mock, interruptBootstrap_mock, start_mock,
                       info_mock, warn_mock, time_mock, sleep_mock):
     shared_state = SharedState("root", "sshkey_file", "scriptDir", "bootdir",