Browse Source

AMBARI-4319. Task timeout should be a configurable knob at the ambari-server (dlysnichenko)

Lisnichenko Dmitro 11 years ago
parent
commit
40e94f00d6

+ 0 - 2
ambari-agent/conf/unix/ambari-agent.ini

@@ -31,8 +31,6 @@ puppetmodules=/var/lib/ambari-agent/puppet
 ruby_home=/usr/lib/ambari-agent/lib/ruby-1.8.7-p370
 puppet_home=/usr/lib/ambari-agent/lib/puppet-2.7.9
 facter_home=/usr/lib/ambari-agent/lib/facter-1.6.10
-# How many seconds will pass before running puppet is terminated on timeout
-timeout_seconds = 600
 
 [python]
 custom_actions_dir = /var/lib/ambari-agent/resources/custom_actions

+ 11 - 6
ambari-agent/src/main/python/ambari_agent/PuppetExecutor.py

@@ -125,17 +125,20 @@ class PuppetExecutor:
   def run_manifest(self, command, file, tmpoutfile, tmperrfile):
     result = {}
     taskId = 0
+    timeout = command ['commandParams']['command_timeout']
     if command.has_key("taskId"):
       taskId = command['taskId']
     puppetEnv = os.environ
     #Install repos
-    repo_manifest_list = self.generate_repo_manifests(command, self.tmpDir, self.modulesdir, taskId)
+    repo_manifest_list = self.generate_repo_manifests(command, self.tmpDir,
+                                                      self.modulesdir, taskId)
     puppetFiles = list(repo_manifest_list)
     puppetFiles.append(file)
     #Run all puppet commands, from manifest generator and for repos installation
     #Appending outputs and errors, exitcode - maximal from all
     for puppetFile in puppetFiles:
-      self.runPuppetFile(puppetFile, result, puppetEnv, tmpoutfile, tmperrfile)
+      self.runPuppetFile(puppetFile, result, puppetEnv, tmpoutfile,
+                         tmperrfile, timeout)
       # Check if one of the puppet command fails and error out
       if not self.isSuccessfull(result["exitcode"]):
         break
@@ -178,7 +181,8 @@ class PuppetExecutor:
       result = {'stdout': '', 'stderr': errMsg, 'exitcode': 1}
     return result
 
-  def runPuppetFile(self, puppetFile, result, puppetEnv, tmpoutfile, tmperrfile):
+  def runPuppetFile(self, puppetFile, result, puppetEnv, tmpoutfile,
+                    tmperrfile, timeout):
     """ Run the command and make sure the output gets propagated"""
     puppetcommand = self.puppetCommand(puppetFile)
     rubyLib = ""
@@ -200,7 +204,8 @@ class PuppetExecutor:
     logger.debug("Launching watchdog thread")
     self.event.clear()
     self.last_puppet_has_been_killed = False
-    thread = Thread(target =  self.puppet_watchdog_func, args = (puppet, ))
+    thread = Thread(target =  self.puppet_watchdog_func,
+                    args = (puppet, timeout))
     thread.start()
     # Waiting for process to finished or killed
     puppet.communicate()
@@ -246,8 +251,8 @@ class PuppetExecutor:
       stderr=tmperr,
       env=puppetEnv)
 
-  def puppet_watchdog_func(self, puppet):
-    self.event.wait(float(self.puppet_timeout))
+  def puppet_watchdog_func(self, puppet, puppet_timeout):
+    self.event.wait(float(puppet_timeout))
     if puppet.returncode is None:
       logger.error("Task timed out, killing process with PID: " + str(puppet.pid))
       shell.kill_process_with_children(puppet.pid)

+ 4 - 0
ambari-agent/src/main/python/ambari_agent/test.json

@@ -33,6 +33,10 @@
 }]
 
 },
+"commandParams":
+{
+  "command_timeout": "600"
+},
 
 
 "configurations" : {

+ 8 - 7
ambari-agent/src/test/python/ambari_agent/TestPuppetExecutor.py

@@ -67,7 +67,7 @@ class TestPuppetExecutor(TestCase):
     jsonStr = jsonFile.read()
     parsedJson = json.loads(jsonStr)
     parsedJson["taskId"] = 1
-    def side_effect1(puppetFile, result, puppetEnv, tmpoutfile, tmperrfile):
+    def side_effect1(puppetFile, result, puppetEnv, tmpoutfile, tmperrfile, timeout):
         result["exitcode"] = 0
     runPuppetFileMock.side_effect = side_effect1
     generateManifestMock.return_value = ''
@@ -77,7 +77,7 @@ class TestPuppetExecutor(TestCase):
     self.assertEquals(res["exitcode"], 0)
     self.assertTrue(puppetInstance.reposInstalled)
 
-    def side_effect2(puppetFile, result, puppetEnv, tmpoutfile, tmperrfile):
+    def side_effect2(puppetFile, result, puppetEnv, tmpoutfile, tmperrfile, timeout):
         result["exitcode"] = 999
     runPuppetFileMock.side_effect = side_effect2
     puppetInstance.reposInstalled = False
@@ -124,7 +124,7 @@ class TestPuppetExecutor(TestCase):
     parsedJson = json.loads(jsonStr)
     parsedJson["taskId"] = 77
     parsedJson['roleCommand'] = "START"
-    def side_effect(puppetFile, result, puppetEnv, tmpoutfile, tmperrfile):
+    def side_effect(puppetFile, result, puppetEnv, tmpoutfile, tmperrfile, timeout):
       result["exitcode"] = 0
     runPuppetFileMock.side_effect = side_effect
     
@@ -223,7 +223,6 @@ class TestPuppetExecutor(TestCase):
     """
     subproc_mock = self.Subprocess_mockup()
     config = AmbariConfig().getConfig()
-    config.set('puppet','timeout_seconds',"0.1")
     executor_mock = self.PuppetExecutor_mock("/home/centos/ambari_repo_info/ambari-agent/src/main/puppet/",
       "/usr/",
       "/root/workspace/puppet-install/facter-1.6.10/",
@@ -234,7 +233,9 @@ class TestPuppetExecutor(TestCase):
     puppetEnv = { "RUBYLIB" : ""}
     kill_process_with_children_mock.side_effect = lambda pid : subproc_mock.terminate()
     subproc_mock.returncode = None
-    thread = Thread(target =  executor_mock.runPuppetFile, args = ("fake_puppetFile", result, puppetEnv, tmpoutfile, tmperrfile))
+    timeout = "0.1"
+    thread = Thread(target =  executor_mock.runPuppetFile, args = ("fake_puppetFile",
+        result, puppetEnv, tmpoutfile, tmperrfile, timeout))
     thread.start()
     time.sleep(0.1)
     subproc_mock.finished_event.wait()
@@ -247,7 +248,6 @@ class TestPuppetExecutor(TestCase):
     """
     subproc_mock = self.Subprocess_mockup()
     config = AmbariConfig().getConfig()
-    config.set('puppet','timeout_seconds',"5")
     executor_mock = self.PuppetExecutor_mock("/home/centos/ambari_repo_info/ambari-agent/src/main/puppet/",
     "/usr/",
     "/root/workspace/puppet-install/facter-1.6.10/",
@@ -257,8 +257,9 @@ class TestPuppetExecutor(TestCase):
     result = {  }
     puppetEnv = { "RUBYLIB" : ""}
     subproc_mock.returncode = 0
+    timeout = "5"
     thread = Thread(target =  executor_mock.runPuppetFile, args = ("fake_puppetFile",
-                            result, puppetEnv, tmpoutfile, tmperrfile))
+                            result, puppetEnv, tmpoutfile, tmperrfile, timeout))
     thread.start()
     time.sleep(0.1)
     subproc_mock.should_finish_event.set()

+ 2 - 0
ambari-server/conf/unix/ambari.properties

@@ -39,3 +39,5 @@ server.execution.scheduler.maxThreads=5
 server.execution.scheduler.maxDbConnections=5
 server.execution.scheduler.misfire.toleration.minutes=480
 
+# Default timeout in seconds before task is killed
+agent.task.timeout=600

+ 2 - 1
ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionScheduler.java

@@ -197,6 +197,7 @@ class ActionScheduler implements Runnable {
           affectedHosts.addAll(stageHosts);
         }
 
+        // Commands that will be scheduled in current scheduler wakeup
         List<ExecutionCommand> commandsToSchedule = new ArrayList<ExecutionCommand>();
         Map<String, RoleStats> roleStats = processInProgressStage(s, commandsToSchedule);
         // Check if stage is failed
@@ -362,7 +363,7 @@ class ActionScheduler implements Runnable {
     long now = System.currentTimeMillis();
     long taskTimeout = actionTimeout;
     if (taskTimeoutAdjustment) {
-      taskTimeout = actionTimeout + s.getTaskTimeout();
+      taskTimeout = actionTimeout + s.getStageTimeout();
     }
     for (String host : s.getHosts()) {
       List<ExecutionCommandWrapper> commandWrappers = s.getExecutionCommands(host);

+ 26 - 14
ambari-server/src/main/java/org/apache/ambari/server/actionmanager/Stage.java

@@ -22,9 +22,7 @@ import java.util.*;
 import org.apache.ambari.server.Role;
 import org.apache.ambari.server.RoleCommand;
 import org.apache.ambari.server.agent.ExecutionCommand;
-import org.apache.ambari.server.orm.dao.HostDAO;
 import org.apache.ambari.server.orm.dao.HostRoleCommandDAO;
-import org.apache.ambari.server.orm.dao.StageDAO;
 import org.apache.ambari.server.orm.entities.HostRoleCommandEntity;
 import org.apache.ambari.server.orm.entities.RoleSuccessCriteriaEntity;
 import org.apache.ambari.server.orm.entities.StageEntity;
@@ -34,7 +32,6 @@ import org.apache.ambari.server.state.svccomphost.ServiceComponentHostUpgradeEve
 import org.apache.ambari.server.utils.StageUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import com.google.inject.Injector;
 import com.google.inject.assistedinject.Assisted;
 import com.google.inject.assistedinject.AssistedInject;
 
@@ -52,8 +49,7 @@ public class Stage {
   private final String requestContext;
   private String clusterHostInfo;
 
-  private int taskTimeout = -1;
-  private int perTaskTimeFactor = 60000;
+  private int stageTimeout = -1;
 
   //Map of roles to successFactors for this stage. Default is 1 i.e. 100%
   private Map<Role, Float> successFactors = new HashMap<Role, Float>();
@@ -450,18 +446,34 @@ public class Stage {
   HostRoleCommand getHostRoleCommand(String hostname, String role) {
     return hostRoleCommands.get(hostname).get(role);
   }
-  
-  public synchronized int getTaskTimeout() {
-    if (taskTimeout == -1) {
-      int maxTasks = 0;
+
+  /**
+   * In this method we sum up all timeout values for all commands inside stage
+   */
+  public synchronized int getStageTimeout() {
+    if (stageTimeout == -1) {
       for (String host: commandsToSend.keySet()) {
-        if (commandsToSend.get(host).size() > maxTasks) {
-          maxTasks = commandsToSend.get(host).size();
+        int summaryTaskTimeoutForHost = 0;
+        for (ExecutionCommandWrapper command : commandsToSend.get(host)) {
+          Map<String, String> commandParams =
+                command.getExecutionCommand().getCommandParams();
+          String timeoutKey = ExecutionCommand.KeyNames.COMMAND_TIMEOUT;
+          if (commandParams != null && commandParams.containsKey(timeoutKey)) {
+            String timeoutStr = commandParams.get(timeoutKey);
+            long commandTimeout =
+                Long.parseLong(timeoutStr) * 1000; // Converting to milliseconds
+            summaryTaskTimeoutForHost += commandTimeout;
+          } else {
+            LOG.error("Execution command has no timeout parameter" +
+                    command.toString());
+          }
+        }
+        if (summaryTaskTimeoutForHost > stageTimeout) {
+          stageTimeout = summaryTaskTimeoutForHost;
         }
       }
-      taskTimeout = maxTasks * perTaskTimeFactor;
-    }  
-    return taskTimeout;
+    }
+    return stageTimeout;
   }
 
   @Override //Object

+ 1 - 2
ambari-server/src/main/java/org/apache/ambari/server/agent/ExecutionCommand.java

@@ -258,9 +258,8 @@ public class ExecutionCommand extends AgentCommand {
     String AMBARI_DB_RCA_DRIVER = "ambari_db_rca_driver";
     String AMBARI_DB_RCA_USERNAME = "ambari_db_rca_username";
     String AMBARI_DB_RCA_PASSWORD = "ambari_db_rca_password";
-    String SERVICE_CHECK = "SERVICE_CHECK"; // TODO: is it standart command? maybe add it to RoleCommand enum?
 
-    String COMMAND_TIMEOUT_DEFAULT = "600"; // TODO: Will be replaced by proper initialization in another jira
+    String SERVICE_CHECK = "SERVICE_CHECK"; // TODO: is it standart command? maybe add it to RoleCommand enum?
     String CUSTOM_COMMAND = "custom_command";
 
   }

+ 7 - 2
ambari-server/src/main/java/org/apache/ambari/server/agent/HeartbeatMonitor.java

@@ -27,6 +27,7 @@ import com.google.inject.Injector;
 import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.actionmanager.ActionManager;
 import org.apache.ambari.server.api.services.AmbariMetaInfo;
+import org.apache.ambari.server.configuration.Configuration;
 import org.apache.ambari.server.state.*;
 import org.apache.ambari.server.state.fsm.InvalidStateTransitionException;
 import org.apache.ambari.server.state.host.HostHeartbeatLostEvent;
@@ -48,6 +49,7 @@ public class HeartbeatMonitor implements Runnable {
   private Thread monitorThread = null;
   private final ConfigHelper configHelper;
   private final AmbariMetaInfo ambariMetaInfo;
+  private final Configuration configuration;
 
   public HeartbeatMonitor(Clusters clusters, ActionQueue aq, ActionManager am,
                           int threadWakeupInterval, Injector injector) {
@@ -57,6 +59,7 @@ public class HeartbeatMonitor implements Runnable {
     this.threadWakeupInterval = threadWakeupInterval;
     this.configHelper = injector.getInstance(ConfigHelper.class);
     this.ambariMetaInfo = injector.getInstance(AmbariMetaInfo.class);
+    this.configuration = injector.getInstance(Configuration.class);
   }
 
   public void shutdown() {
@@ -241,13 +244,15 @@ public class HeartbeatMonitor implements Runnable {
     Map<String, String> commandParams = statusCmd.getCommandParams();
     commandParams.put(SCHEMA_VERSION, serviceInfo.getSchemaVersion());
 
-    String commandTimeout = ExecutionCommand.KeyNames.COMMAND_TIMEOUT_DEFAULT;
+    String commandTimeout = configuration.getDefaultAgentTaskTimeout();
     CommandScriptDefinition script = componentInfo.getCommandScript();
     if (serviceInfo.getSchemaVersion().equals(AmbariMetaInfo.SCHEMA_VERSION_2)) {
       if (script != null) {
         commandParams.put(SCRIPT, script.getScript());
         commandParams.put(SCRIPT_TYPE, script.getScriptType().toString());
-        commandTimeout = String.valueOf(script.getTimeout());
+        if (script.getTimeout() > 0) {
+          commandTimeout = String.valueOf(script.getTimeout());
+        }
       } else {
         String message = String.format("Component %s of service %s has not " +
                 "command script defined", componentName, serviceName);

+ 19 - 0
ambari-server/src/main/java/org/apache/ambari/server/configuration/Configuration.java

@@ -289,6 +289,9 @@ public class Configuration {
           "server.stages.parallel";
   private static final String PARALLEL_STAGE_EXECUTION_DEFAULT = "true";
 
+  public static final String AGENT_TASK_TIMEOUT_KEY = "agent.task.timeout";
+  public static final String AGENT_TASK_TIMEOUT_DEFAULT = "600";
+
   private static final Logger LOG = LoggerFactory.getLogger(
       Configuration.class);
 
@@ -926,4 +929,20 @@ public class Configuration {
     return "true".equalsIgnoreCase(configsMap.get(PARALLEL_STAGE_EXECUTION_KEY));
   }
 
+  /**
+   * @return default task timeout in seconds (string representation). This value
+   * is used at python (agent) code.
+   */
+  public String getDefaultAgentTaskTimeout() {
+    String value =  properties.getProperty(AGENT_TASK_TIMEOUT_KEY, AGENT_TASK_TIMEOUT_DEFAULT);
+    if (StringUtils.isNumeric(value)) {
+      return value;
+    } else {
+      LOG.warn(String.format("Value of %s (%s) should be a number, " +
+              "falling back to default value (%s)",
+              AGENT_TASK_TIMEOUT_KEY, value, AGENT_TASK_TIMEOUT_DEFAULT));
+      return AGENT_TASK_TIMEOUT_DEFAULT;
+    }
+  }
+
 }

+ 14 - 9
ambari-server/src/main/java/org/apache/ambari/server/controller/AmbariCustomCommandExecutionHelper.java

@@ -59,7 +59,6 @@ import java.util.Set;
 import java.util.TreeMap;
 
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.COMMAND_TIMEOUT;
-import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.COMMAND_TIMEOUT_DEFAULT;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.CUSTOM_COMMAND;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.DB_DRIVER_FILENAME;
 import static org.apache.ambari.server.agent.ExecutionCommand.KeyNames.DB_NAME;
@@ -248,7 +247,7 @@ public class AmbariCustomCommandExecutionHelper {
       Map<String, String> commandParams = new TreeMap<String, String>();
       commandParams.put(SCHEMA_VERSION, serviceInfo.getSchemaVersion());
 
-      String commandTimeout = COMMAND_TIMEOUT_DEFAULT;
+      String commandTimeout = configs.getDefaultAgentTaskTimeout();
 
       if (serviceInfo.getSchemaVersion().equals(AmbariMetaInfo.SCHEMA_VERSION_2)) {
         // Service check command is not custom command
@@ -260,11 +259,13 @@ public class AmbariCustomCommandExecutionHelper {
         if (script != null) {
           commandParams.put(SCRIPT, script.getScript());
           commandParams.put(SCRIPT_TYPE, script.getScriptType().toString());
-          commandTimeout = String.valueOf(script.getTimeout());
+          if (script.getTimeout() > 0) {
+            commandTimeout = String.valueOf(script.getTimeout());
+          }
         } else {
           String message = String.format("Component %s has not command script " +
-              "defined. It is not possible to run service check" +
-              " for this service", componentName);
+              "defined. It is not possible to send command for " +
+                  "this service", componentName);
           throw new AmbariException(message);
         }
         // We don't need package/repo infomation to perform service check
@@ -382,7 +383,7 @@ public class AmbariCustomCommandExecutionHelper {
     Map<String, String> commandParams = new TreeMap<String, String>();
     commandParams.put(SCHEMA_VERSION, serviceInfo.getSchemaVersion());
 
-    String commandTimeout = COMMAND_TIMEOUT_DEFAULT;
+    String commandTimeout = configs.getDefaultAgentTaskTimeout();
 
 
     if (serviceInfo.getSchemaVersion().equals(AmbariMetaInfo.SCHEMA_VERSION_2)) {
@@ -391,7 +392,9 @@ public class AmbariCustomCommandExecutionHelper {
       if (script != null) {
         commandParams.put(SCRIPT, script.getScript());
         commandParams.put(SCRIPT_TYPE, script.getScriptType().toString());
-        commandTimeout = String.valueOf(script.getTimeout());
+        if (script.getTimeout() > 0) {
+          commandTimeout = String.valueOf(script.getTimeout());
+        }
       } else {
         String message = String.format("Service %s has no command script " +
             "defined. It is not possible to run service check" +
@@ -569,13 +572,15 @@ public class AmbariCustomCommandExecutionHelper {
      * component main commandScript to agent. This script is only used for
      * default commads like INSTALL/STOP/START/CONFIGURE
      */
-    String commandTimeout = ExecutionCommand.KeyNames.COMMAND_TIMEOUT_DEFAULT;
+    String commandTimeout =configs.getDefaultAgentTaskTimeout();
     CommandScriptDefinition script = componentInfo.getCommandScript();
     if (serviceInfo.getSchemaVersion().equals(AmbariMetaInfo.SCHEMA_VERSION_2)) {
       if (script != null) {
         commandParams.put(SCRIPT, script.getScript());
         commandParams.put(SCRIPT_TYPE, script.getScriptType().toString());
-        commandTimeout = String.valueOf(script.getTimeout());
+        if (script.getTimeout() > 0) {
+          commandTimeout = String.valueOf(script.getTimeout());
+        }
       } else {
         String message = String.format("Component %s of service %s has no " +
             "command script defined", componentName, serviceName);

+ 6 - 1
ambari-server/src/main/java/org/apache/ambari/server/controller/ControllerModule.java

@@ -108,7 +108,12 @@ public class ControllerModule extends AbstractModule {
     bind(ActionDBAccessor.class).to(ActionDBAccessorImpl.class);
     bind(CustomActionDBAccessor.class).to(CustomActionDBAccessorImpl.class);
     bindConstant().annotatedWith(Names.named("schedulerSleeptime")).to(10000L);
-    bindConstant().annotatedWith(Names.named("actionTimeout")).to(600000L);
+
+    // This time is added to summary timeout time of all tasks in stage
+    // So it's an "additional time", given to stage to finish execution before
+    // it is considered as timed out
+    bindConstant().annotatedWith(Names.named("actionTimeout")).to(120000L);
+
     bindConstant().annotatedWith(Names.named("dbInitNeeded")).to(dbInitNeeded);
     bindConstant().annotatedWith(Names.named("statusCheckInterval")).to(5000L);
 

+ 3 - 2
ambari-server/src/main/java/org/apache/ambari/server/state/CommandScriptDefinition.java

@@ -36,9 +36,10 @@ public class CommandScriptDefinition {
   private Type scriptType = Type.PYTHON;
 
   /**
-   * Timeout is given in seconds
+   * Timeout is given in seconds. Default value of 0 is used if not
+   * overridden at xml file
    */
-  private int timeout = 600;
+  private int timeout = 0;
 
 
   public String getScript() {

+ 3 - 0
ambari-server/src/main/java/org/apache/ambari/server/utils/StageUtils.java

@@ -174,6 +174,9 @@ public class StageUtils {
     Map<String, String> roleParams = new TreeMap<String, String>();
     roleParams.put("format", "false");
     execCmd.setRoleParams(roleParams);
+    Map<String, String> commandParams = new TreeMap<String, String>();
+    commandParams.put(ExecutionCommand.KeyNames.COMMAND_TIMEOUT, "600");
+    execCmd.setCommandParams(commandParams);
     return s;
   }
 

+ 10 - 1
ambari-server/src/test/java/org/apache/ambari/server/actionmanager/TestStage.java

@@ -21,9 +21,13 @@ import static org.junit.Assert.*;
 
 import org.apache.ambari.server.Role;
 import org.apache.ambari.server.RoleCommand;
+import org.apache.ambari.server.agent.ExecutionCommand;
 import org.apache.ambari.server.utils.StageUtils;
 import org.junit.Test;
 
+import java.util.Map;
+import java.util.TreeMap;
+
 public class TestStage {
 
   private static final String CLUSTER_HOST_INFO = "cluster_host_info";
@@ -35,7 +39,12 @@ public class TestStage {
         null, "c1", "HDFS");
     s.addHostRoleExecutionCommand("h1", Role.HBASE_MASTER, RoleCommand.INSTALL,
         null, "c1", "HBASE");
-    assertEquals(3*60000, s.getTaskTimeout());
+    for (ExecutionCommandWrapper wrapper : s.getExecutionCommands("h1")) {
+      Map<String, String> commandParams = new TreeMap<String, String>();
+      commandParams.put(ExecutionCommand.KeyNames.COMMAND_TIMEOUT, "600");
+      wrapper.getExecutionCommand().setCommandParams(commandParams);
+    }
+    assertEquals(3*600000, s.getStageTimeout());
   }
 
   @Test

+ 8 - 2
ambari-server/src/test/java/org/apache/ambari/server/api/util/StackExtensionHelperTest.java

@@ -100,7 +100,7 @@ public class StackExtensionHelperTest {
                 firstComponent.getCommandScript().getScript());
         assertEquals(CommandScriptDefinition.Type.PYTHON,
                 firstComponent.getCommandScript().getScriptType());
-        assertEquals(600,
+        assertEquals(777,
                 firstComponent.getCommandScript().getTimeout());
         // Check custom commands for component
         List<CustomCommandDefinition> customCommands =
@@ -112,7 +112,7 @@ public class StackExtensionHelperTest {
                 customCommands.get(0).getCommandScript().getScript());
         assertEquals(CommandScriptDefinition.Type.PYTHON,
                 customCommands.get(0).getCommandScript().getScriptType());
-        assertEquals(600,
+        assertEquals(888,
                 customCommands.get(0).getCommandScript().getTimeout());
         // Check all parsed os specifics
         Map<String,ServiceOsSpecific> specifics = serviceInfo.getOsSpecifics();
@@ -122,6 +122,12 @@ public class StackExtensionHelperTest {
         assertEquals("rpm", anyOs.getPackages().get(0).getType());
         assertEquals("wget", anyOs.getPackages().get(0).getName());
 
+        // Test default timeout value
+        ComponentInfo secondComponent = components.get(1);
+        assertEquals("HBASE_REGIONSERVER", secondComponent.getName());
+        assertEquals(0,
+                secondComponent.getCommandScript().getTimeout());
+
         ServiceOsSpecific c6Os = specifics.get("centos6");
         assertEquals("centos6", c6Os.getOsType());
         assertEquals("rpm", c6Os.getPackages().get(0).getType());

+ 2 - 2
ambari-server/src/test/resources/stacks/HDP/2.0.7/services/HBASE/metainfo.xml

@@ -34,7 +34,7 @@
           <commandScript> <!--This is the script to handle all default commands -->
             <script>scripts/hbase_master.py</script>
             <scriptType>PYTHON</scriptType>
-            <timeout>600</timeout> <!--You can also use the default value here-->
+            <timeout>777</timeout> <!--You can also use the default value here-->
           </commandScript>
           <customCommands>
             <customCommand> <!--Specialized only for custom commands, can use the same script as well -->
@@ -42,7 +42,7 @@
               <commandScript>
                 <script>scripts/hbase_master_restart.py</script>
                 <scriptType>PYTHON</scriptType>
-                <timeout>600</timeout>
+                <timeout>888</timeout>
               </commandScript>
             </customCommand>
           </customCommands>