You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@ambari.apache.org by Alejandro Fernandez <af...@hortonworks.com> on 2016/03/16 21:41:01 UTC

Review Request 44926: [DRAFT] Auto-retry on failure during RU/EU

-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/44926/
-----------------------------------------------------------

Review request for Ambari, Jonathan Hurley and Nate Cole.


Bugs: AMBARI-15446
    https://issues.apache.org/jira/browse/AMBARI-15446


Repository: ambari


Description
-------

When a failure occurs during RU/EU and the task transitions to HOLDING_FAILED or HOLDING_TIMEDOUT, want Ambari to automatically retry up to up to x mins. This is useful when a host goes down as Ambari is running a task on it.
ambari.properties will have 1 new parameter. E.g,. 
stack-upgrade.max_retry_timeout_mins=15 (by default, will not be present)
If Ambari Server is restarted, it should be able to recover.
Today, Action Scheduler increases the attempt_count whenever a task is retried, but it requires resetting the start_time to -1. Because of this, we cannot rely on the start_time property to know when to timeout after several retries.

For the implementation, will add another thread to Ambari that will monitor failed tasks only during active RU/EU and change the status back to PENDING so that Action Scheduler can reschedule it.
Luckily, any tasks in HOLDING_TIMEDOUT and HOLDING_FAILED states are blocking, so no other stages are allowed to proceed.
In order to know when a task was first started, will add a new property to host_role_command table called original_start_time.

For the agents, we need to ensure that they always write out a response. On the first heartbeat, it should send the status of its last command so we know it failed and Ambari can retry.


Diffs
-----

  ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java 3a80803 
  ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java PRE-CREATION 
  ambari-server/src/main/java/org/apache/ambari/server/checks/PreviousUpgradeCompleted.java 3a4467f 
  ambari-server/src/main/java/org/apache/ambari/server/orm/dao/ClusterVersionDAO.java 1bcca60 
  ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostRoleCommandDAO.java f5b1cb4 
  ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ClusterVersionEntity.java f1867b4 
  ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostRoleCommandEntity.java 19f0602 
  ambari-server/src/main/java/org/apache/ambari/server/state/Cluster.java ed3c772 
  ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClusterImpl.java 1c7ff61 
  ambari-server/src/main/java/org/apache/ambari/server/topology/LogicalRequest.java 82edbcf 

Diff: https://reviews.apache.org/r/44926/diff/


Testing
-------

Verified on a live cluster.

TODO: Still need to make more changes to the implementation, add the config, switch to gauva service, add a column, and add unit tests.


Thanks,

Alejandro Fernandez


Re: Review Request 44926: [DRAFT] Auto-retry on failure during RU/EU

Posted by Jonathan Hurley <jh...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/44926/#review124202
-----------------------------------------------------------




ambari-server/src/main/java/org/apache/ambari/server/actionmanager/HostRoleCommand.java (lines 334 - 340)
<https://reviews.apache.org/r/44926/#comment186640>

    Documentation.



ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java (line 130)
<https://reviews.apache.org/r/44926/#comment186641>

    Get rid of import.



ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (line 51)
<https://reviews.apache.org/r/44926/#comment186650>

    This class is really scoped for stack upgrades. Yet it's name doesn't indicate that nor is it in an upgrade package.
    
    Thoughts on changing the name or moving its package?



ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (line 72)
<https://reviews.apache.org/r/44926/#comment186651>

    Comment on why these are skipped (for those of us who don't have upgrade blood in our veins)



ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (line 84)
<https://reviews.apache.org/r/44926/#comment186642>

    Consistency in name with `m_`



ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (line 89)
<https://reviews.apache.org/r/44926/#comment186643>

    Consistency in name with `m_`



ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (line 105)
<https://reviews.apache.org/r/44926/#comment186645>

    Expose via Configuration & ambari.properties



ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (line 119)
<https://reviews.apache.org/r/44926/#comment186646>

    Log.info that it's not going to run, and why.



ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (line 133)
<https://reviews.apache.org/r/44926/#comment186647>

    Use {} for log statements, especially debug ones.



ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (line 178)
<https://reviews.apache.org/r/44926/#comment186653>

    Can we fail fast for things like this?
    
    ```
    if( null == requestId )
      return
    ```
    
    Makes multiple if/for/if indents easier to read.



ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (line 180)
<https://reviews.apache.org/r/44926/#comment186654>

    Doesn't the contract ensure a non-null collection? You can eliminate this if-statement in that case.



ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (line 207)
<https://reviews.apache.org/r/44926/#comment186652>

    Doc.



ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (line 225)
<https://reviews.apache.org/r/44926/#comment186655>

    Doc



ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostRoleCommandDAO.java (lines 317 - 320)
<https://reviews.apache.org/r/44926/#comment186649>

    Can you use named parameters here and make this a NamedQuery?


- Jonathan Hurley


On March 17, 2016, 7:07 p.m., Alejandro Fernandez wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/44926/
> -----------------------------------------------------------
> 
> (Updated March 17, 2016, 7:07 p.m.)
> 
> 
> Review request for Ambari, Jonathan Hurley and Nate Cole.
> 
> 
> Bugs: AMBARI-15446
>     https://issues.apache.org/jira/browse/AMBARI-15446
> 
> 
> Repository: ambari
> 
> 
> Description
> -------
> 
> When a failure occurs during RU/EU and the task transitions to HOLDING_FAILED or HOLDING_TIMEDOUT, want Ambari to automatically retry up to up to x mins. This is useful when a host goes down as Ambari is running a task on it.
> ambari.properties will have 1 new parameter. E.g,. 
> stack-upgrade.max_retry_timeout_mins=15 (by default, will not be present)
> If Ambari Server is restarted, it should be able to recover.
> Today, Action Scheduler increases the attempt_count whenever a task is retried, but it requires resetting the start_time to -1. Because of this, we cannot rely on the start_time property to know when to timeout after several retries.
> 
> For the implementation, will add another thread to Ambari that will monitor failed tasks only during active RU/EU and change the status back to PENDING so that Action Scheduler can reschedule it.
> Luckily, any tasks in HOLDING_TIMEDOUT and HOLDING_FAILED states are blocking, so no other stages are allowed to proceed.
> In order to know when a task was first started, will add a new property to host_role_command table called original_start_time.
> 
> For the agents, we need to ensure that they always write out a response. On the first heartbeat, it should send the status of its last command so we know it failed and Ambari can retry.
> 
> 
> Diffs
> -----
> 
>   ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionDBAccessorImpl.java 429f573 
>   ambari-server/src/main/java/org/apache/ambari/server/actionmanager/HostRoleCommand.java 2764b3f 
>   ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java 3a80803 
>   ambari-server/src/main/java/org/apache/ambari/server/agent/HeartbeatProcessor.java a1a686a 
>   ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java PRE-CREATION 
>   ambari-server/src/main/java/org/apache/ambari/server/configuration/Configuration.java 9404506 
>   ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostRoleCommandDAO.java f5b1cb4 
>   ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostRoleCommandEntity.java 19f0602 
>   ambari-server/src/main/java/org/apache/ambari/server/topology/HostRequest.java 9eb514a 
>   ambari-server/src/main/java/org/apache/ambari/server/topology/LogicalRequest.java 82edbcf 
>   ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog240.java a803f73 
>   ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql 9b4810c 
>   ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql cc3d197 
>   ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql 07c786d 
>   ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql ab6dc93 
>   ambari-server/src/main/resources/Ambari-DDL-SQLAnywhere-CREATE.sql 8e91fde 
>   ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql 440ca44 
> 
> Diff: https://reviews.apache.org/r/44926/diff/
> 
> 
> Testing
> -------
> 
> Verified on a live cluster.
> 
> TODO: Still need to make more changes to the implementation, add the config, switch to gauva service, add a column, and add unit tests.
> 
> 
> Thanks,
> 
> Alejandro Fernandez
> 
>


Re: Review Request 44926: Auto-retry on failure during RU/EU

Posted by Alejandro Fernandez <af...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/44926/
-----------------------------------------------------------

(Updated March 21, 2016, 8:59 p.m.)


Review request for Ambari, Jonathan Hurley and Nate Cole.


Bugs: AMBARI-15446
    https://issues.apache.org/jira/browse/AMBARI-15446


Repository: ambari


Description
-------

When a failure occurs during RU/EU and the task transitions to HOLDING_FAILED or HOLDING_TIMEDOUT, want Ambari to automatically retry up to up to x mins. This is useful when a host goes down as Ambari is running a task on it.
ambari.properties will have 1 new parameter. E.g,. 
stack-upgrade.max_retry_timeout_mins=15 (by default, will not be present)
If Ambari Server is restarted, it should be able to recover.
Today, Action Scheduler increases the attempt_count whenever a task is retried, but it requires resetting the start_time to -1. Because of this, we cannot rely on the start_time property to know when to timeout after several retries.

For the implementation, will add another thread to Ambari that will monitor failed tasks only during active RU/EU and change the status back to PENDING so that Action Scheduler can reschedule it.
Luckily, any tasks in HOLDING_TIMEDOUT and HOLDING_FAILED states are blocking, so no other stages are allowed to proceed.
In order to know when a task was first started, will add a new property to host_role_command table called original_start_time.

For the agents, we need to ensure that they always write out a response. On the first heartbeat, it should send the status of its last command so we know it failed and Ambari can retry.


Diffs
-----

  ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionDBAccessorImpl.java 429f573 
  ambari-server/src/main/java/org/apache/ambari/server/actionmanager/HostRoleCommand.java 2764b3f 
  ambari-server/src/main/java/org/apache/ambari/server/agent/HeartbeatProcessor.java a1a686a 
  ambari-server/src/main/java/org/apache/ambari/server/configuration/Configuration.java 9404506 
  ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostRoleCommandDAO.java f5b1cb4 
  ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostRoleCommandEntity.java 19f0602 
  ambari-server/src/main/java/org/apache/ambari/server/state/services/RetryUpgradeActionService.java PRE-CREATION 
  ambari-server/src/main/java/org/apache/ambari/server/topology/HostRequest.java 9eb514a 
  ambari-server/src/main/java/org/apache/ambari/server/topology/LogicalRequest.java 82edbcf 
  ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog240.java 7b83710 
  ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql a07c6fc 
  ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql b2b450a 
  ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql cec122e 
  ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql 96fc720 
  ambari-server/src/main/resources/Ambari-DDL-SQLAnywhere-CREATE.sql c425d6f 
  ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql 2a89e26 
  ambari-server/src/test/java/org/apache/ambari/server/state/services/RetryUpgradeActionServiceTest.java PRE-CREATION 

Diff: https://reviews.apache.org/r/44926/diff/


Testing (updated)
-------

Verified on a live cluster.
Unit tests passed,

mvn clean package test

[INFO] ------------------------------------------------------------------------
[INFO] BUILD SUCCESS
[INFO] ------------------------------------------------------------------------
[INFO] Total time: 01:11 h
[INFO] Finished at: 2016-03-21T13:15:21-07:00
[INFO] Final Memory: 139M/4054M
[INFO] ------------------------------------------------------------------------


Thanks,

Alejandro Fernandez


Re: Review Request 44926: Auto-retry on failure during RU/EU

Posted by Jonathan Hurley <jh...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/44926/#review124629
-----------------------------------------------------------


Ship it!




Ship It!

- Jonathan Hurley


On March 21, 2016, 3:03 p.m., Alejandro Fernandez wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/44926/
> -----------------------------------------------------------
> 
> (Updated March 21, 2016, 3:03 p.m.)
> 
> 
> Review request for Ambari, Jonathan Hurley and Nate Cole.
> 
> 
> Bugs: AMBARI-15446
>     https://issues.apache.org/jira/browse/AMBARI-15446
> 
> 
> Repository: ambari
> 
> 
> Description
> -------
> 
> When a failure occurs during RU/EU and the task transitions to HOLDING_FAILED or HOLDING_TIMEDOUT, want Ambari to automatically retry up to up to x mins. This is useful when a host goes down as Ambari is running a task on it.
> ambari.properties will have 1 new parameter. E.g,. 
> stack-upgrade.max_retry_timeout_mins=15 (by default, will not be present)
> If Ambari Server is restarted, it should be able to recover.
> Today, Action Scheduler increases the attempt_count whenever a task is retried, but it requires resetting the start_time to -1. Because of this, we cannot rely on the start_time property to know when to timeout after several retries.
> 
> For the implementation, will add another thread to Ambari that will monitor failed tasks only during active RU/EU and change the status back to PENDING so that Action Scheduler can reschedule it.
> Luckily, any tasks in HOLDING_TIMEDOUT and HOLDING_FAILED states are blocking, so no other stages are allowed to proceed.
> In order to know when a task was first started, will add a new property to host_role_command table called original_start_time.
> 
> For the agents, we need to ensure that they always write out a response. On the first heartbeat, it should send the status of its last command so we know it failed and Ambari can retry.
> 
> 
> Diffs
> -----
> 
>   ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionDBAccessorImpl.java 429f573 
>   ambari-server/src/main/java/org/apache/ambari/server/actionmanager/HostRoleCommand.java 2764b3f 
>   ambari-server/src/main/java/org/apache/ambari/server/agent/HeartbeatProcessor.java a1a686a 
>   ambari-server/src/main/java/org/apache/ambari/server/configuration/Configuration.java 9404506 
>   ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostRoleCommandDAO.java f5b1cb4 
>   ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostRoleCommandEntity.java 19f0602 
>   ambari-server/src/main/java/org/apache/ambari/server/state/services/RetryUpgradeActionService.java PRE-CREATION 
>   ambari-server/src/main/java/org/apache/ambari/server/topology/HostRequest.java 9eb514a 
>   ambari-server/src/main/java/org/apache/ambari/server/topology/LogicalRequest.java 82edbcf 
>   ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog240.java 7b83710 
>   ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql a07c6fc 
>   ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql b2b450a 
>   ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql cec122e 
>   ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql 96fc720 
>   ambari-server/src/main/resources/Ambari-DDL-SQLAnywhere-CREATE.sql c425d6f 
>   ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql 2a89e26 
>   ambari-server/src/test/java/org/apache/ambari/server/state/services/RetryUpgradeActionServiceTest.java PRE-CREATION 
> 
> Diff: https://reviews.apache.org/r/44926/diff/
> 
> 
> Testing
> -------
> 
> Verified on a live cluster.
> New unit test passed, waiting for full set of unit test results.
> 
> 
> Thanks,
> 
> Alejandro Fernandez
> 
>


Re: Review Request 44926: Auto-retry on failure during RU/EU

Posted by Alejandro Fernandez <af...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/44926/
-----------------------------------------------------------

(Updated March 21, 2016, 7:03 p.m.)


Review request for Ambari, Jonathan Hurley and Nate Cole.


Bugs: AMBARI-15446
    https://issues.apache.org/jira/browse/AMBARI-15446


Repository: ambari


Description
-------

When a failure occurs during RU/EU and the task transitions to HOLDING_FAILED or HOLDING_TIMEDOUT, want Ambari to automatically retry up to up to x mins. This is useful when a host goes down as Ambari is running a task on it.
ambari.properties will have 1 new parameter. E.g,. 
stack-upgrade.max_retry_timeout_mins=15 (by default, will not be present)
If Ambari Server is restarted, it should be able to recover.
Today, Action Scheduler increases the attempt_count whenever a task is retried, but it requires resetting the start_time to -1. Because of this, we cannot rely on the start_time property to know when to timeout after several retries.

For the implementation, will add another thread to Ambari that will monitor failed tasks only during active RU/EU and change the status back to PENDING so that Action Scheduler can reschedule it.
Luckily, any tasks in HOLDING_TIMEDOUT and HOLDING_FAILED states are blocking, so no other stages are allowed to proceed.
In order to know when a task was first started, will add a new property to host_role_command table called original_start_time.

For the agents, we need to ensure that they always write out a response. On the first heartbeat, it should send the status of its last command so we know it failed and Ambari can retry.


Diffs (updated)
-----

  ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionDBAccessorImpl.java 429f573 
  ambari-server/src/main/java/org/apache/ambari/server/actionmanager/HostRoleCommand.java 2764b3f 
  ambari-server/src/main/java/org/apache/ambari/server/agent/HeartbeatProcessor.java a1a686a 
  ambari-server/src/main/java/org/apache/ambari/server/configuration/Configuration.java 9404506 
  ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostRoleCommandDAO.java f5b1cb4 
  ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostRoleCommandEntity.java 19f0602 
  ambari-server/src/main/java/org/apache/ambari/server/state/services/RetryUpgradeActionService.java PRE-CREATION 
  ambari-server/src/main/java/org/apache/ambari/server/topology/HostRequest.java 9eb514a 
  ambari-server/src/main/java/org/apache/ambari/server/topology/LogicalRequest.java 82edbcf 
  ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog240.java 7b83710 
  ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql a07c6fc 
  ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql b2b450a 
  ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql cec122e 
  ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql 96fc720 
  ambari-server/src/main/resources/Ambari-DDL-SQLAnywhere-CREATE.sql c425d6f 
  ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql 2a89e26 
  ambari-server/src/test/java/org/apache/ambari/server/state/services/RetryUpgradeActionServiceTest.java PRE-CREATION 

Diff: https://reviews.apache.org/r/44926/diff/


Testing
-------

Verified on a live cluster.
New unit test passed, waiting for full set of unit test results.


Thanks,

Alejandro Fernandez


Re: Review Request 44926: Auto-retry on failure during RU/EU

Posted by Alejandro Fernandez <af...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/44926/
-----------------------------------------------------------

(Updated March 21, 2016, 6:57 p.m.)


Review request for Ambari, Jonathan Hurley and Nate Cole.


Changes
-------

Addressed comments and retested.


Summary (updated)
-----------------

Auto-retry on failure during RU/EU


Bugs: AMBARI-15446
    https://issues.apache.org/jira/browse/AMBARI-15446


Repository: ambari


Description
-------

When a failure occurs during RU/EU and the task transitions to HOLDING_FAILED or HOLDING_TIMEDOUT, want Ambari to automatically retry up to up to x mins. This is useful when a host goes down as Ambari is running a task on it.
ambari.properties will have 1 new parameter. E.g,. 
stack-upgrade.max_retry_timeout_mins=15 (by default, will not be present)
If Ambari Server is restarted, it should be able to recover.
Today, Action Scheduler increases the attempt_count whenever a task is retried, but it requires resetting the start_time to -1. Because of this, we cannot rely on the start_time property to know when to timeout after several retries.

For the implementation, will add another thread to Ambari that will monitor failed tasks only during active RU/EU and change the status back to PENDING so that Action Scheduler can reschedule it.
Luckily, any tasks in HOLDING_TIMEDOUT and HOLDING_FAILED states are blocking, so no other stages are allowed to proceed.
In order to know when a task was first started, will add a new property to host_role_command table called original_start_time.

For the agents, we need to ensure that they always write out a response. On the first heartbeat, it should send the status of its last command so we know it failed and Ambari can retry.


Diffs (updated)
-----

  ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionDBAccessorImpl.java 429f573 
  ambari-server/src/main/java/org/apache/ambari/server/actionmanager/HostRoleCommand.java 2764b3f 
  ambari-server/src/main/java/org/apache/ambari/server/agent/HeartbeatProcessor.java a1a686a 
  ambari-server/src/main/java/org/apache/ambari/server/configuration/Configuration.java 9404506 
  ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostRoleCommandDAO.java f5b1cb4 
  ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostRoleCommandEntity.java 19f0602 
  ambari-server/src/main/java/org/apache/ambari/server/state/services/RetryUpgradeActionService.java PRE-CREATION 
  ambari-server/src/main/java/org/apache/ambari/server/topology/HostRequest.java 9eb514a 
  ambari-server/src/main/java/org/apache/ambari/server/topology/LogicalRequest.java 82edbcf 
  ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog240.java 7b83710 
  ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql a07c6fc 
  ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql b2b450a 
  ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql cec122e 
  ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql 96fc720 
  ambari-server/src/main/resources/Ambari-DDL-SQLAnywhere-CREATE.sql c425d6f 
  ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql 2a89e26 
  ambari-server/src/test/java/org/apache/ambari/server/state/services/RetryUpgradeActionServiceTest.java PRE-CREATION 

Diff: https://reviews.apache.org/r/44926/diff/


Testing (updated)
-------

Verified on a live cluster.
New unit test passed, waiting for full set of unit test results.


Thanks,

Alejandro Fernandez


Re: Review Request 44926: [DRAFT] Auto-retry on failure during RU/EU

Posted by Alejandro Fernandez <af...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/44926/#review124117
-----------------------------------------------------------




ambari-server/src/main/java/org/apache/ambari/server/actionmanager/HostRoleCommand.java (line 61)
<https://reviews.apache.org/r/44926/#comment186496>

    Today, startTime is allowed to be changed to -1 after it already had value. For this reason, I added a column called originalStartTime that can only be set once.



ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (line 51)
<https://reviews.apache.org/r/44926/#comment186497>

    Switched to guava service :-)



ambari-server/src/main/java/org/apache/ambari/server/configuration/Configuration.java (line 1123)
<https://reviews.apache.org/r/44926/#comment186498>

    Feature will be turned off by default.



ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog240.java (line 643)
<https://reviews.apache.org/r/44926/#comment186499>

    Added column on ambari upgrade.


- Alejandro Fernandez


On March 17, 2016, 11:07 p.m., Alejandro Fernandez wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/44926/
> -----------------------------------------------------------
> 
> (Updated March 17, 2016, 11:07 p.m.)
> 
> 
> Review request for Ambari, Jonathan Hurley and Nate Cole.
> 
> 
> Bugs: AMBARI-15446
>     https://issues.apache.org/jira/browse/AMBARI-15446
> 
> 
> Repository: ambari
> 
> 
> Description
> -------
> 
> When a failure occurs during RU/EU and the task transitions to HOLDING_FAILED or HOLDING_TIMEDOUT, want Ambari to automatically retry up to up to x mins. This is useful when a host goes down as Ambari is running a task on it.
> ambari.properties will have 1 new parameter. E.g,. 
> stack-upgrade.max_retry_timeout_mins=15 (by default, will not be present)
> If Ambari Server is restarted, it should be able to recover.
> Today, Action Scheduler increases the attempt_count whenever a task is retried, but it requires resetting the start_time to -1. Because of this, we cannot rely on the start_time property to know when to timeout after several retries.
> 
> For the implementation, will add another thread to Ambari that will monitor failed tasks only during active RU/EU and change the status back to PENDING so that Action Scheduler can reschedule it.
> Luckily, any tasks in HOLDING_TIMEDOUT and HOLDING_FAILED states are blocking, so no other stages are allowed to proceed.
> In order to know when a task was first started, will add a new property to host_role_command table called original_start_time.
> 
> For the agents, we need to ensure that they always write out a response. On the first heartbeat, it should send the status of its last command so we know it failed and Ambari can retry.
> 
> 
> Diffs
> -----
> 
>   ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionDBAccessorImpl.java 429f573 
>   ambari-server/src/main/java/org/apache/ambari/server/actionmanager/HostRoleCommand.java 2764b3f 
>   ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java 3a80803 
>   ambari-server/src/main/java/org/apache/ambari/server/agent/HeartbeatProcessor.java a1a686a 
>   ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java PRE-CREATION 
>   ambari-server/src/main/java/org/apache/ambari/server/configuration/Configuration.java 9404506 
>   ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostRoleCommandDAO.java f5b1cb4 
>   ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostRoleCommandEntity.java 19f0602 
>   ambari-server/src/main/java/org/apache/ambari/server/topology/HostRequest.java 9eb514a 
>   ambari-server/src/main/java/org/apache/ambari/server/topology/LogicalRequest.java 82edbcf 
>   ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog240.java a803f73 
>   ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql 9b4810c 
>   ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql cc3d197 
>   ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql 07c786d 
>   ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql ab6dc93 
>   ambari-server/src/main/resources/Ambari-DDL-SQLAnywhere-CREATE.sql 8e91fde 
>   ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql 440ca44 
> 
> Diff: https://reviews.apache.org/r/44926/diff/
> 
> 
> Testing
> -------
> 
> Verified on a live cluster.
> 
> TODO: Still need to make more changes to the implementation, add the config, switch to gauva service, add a column, and add unit tests.
> 
> 
> Thanks,
> 
> Alejandro Fernandez
> 
>


Re: Review Request 44926: [DRAFT] Auto-retry on failure during RU/EU

Posted by Nate Cole <nc...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/44926/#review124173
-----------------------------------------------------------


Fix it, then Ship it!




I like it!  Just a couple of nits and a comment.


ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (lines 164 - 165)
<https://reviews.apache.org/r/44926/#comment186608>

    Use {} syntax for logging



ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (lines 211 - 216)
<https://reviews.apache.org/r/44926/#comment186609>

    Not a huge fan of string comparisons here.  Can we ignore server side actions (no host) instead or something?  We should probably do that anyway, and if we have command detail OR command names that specifically need it, let's add an ambari.properties CSV for it.



ambari-server/src/main/java/org/apache/ambari/server/configuration/Configuration.java (line 1123)
<https://reviews.apache.org/r/44926/#comment186610>

    NumberUtils.toInt() can be your friend :)


- Nate Cole


On March 17, 2016, 7:07 p.m., Alejandro Fernandez wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/44926/
> -----------------------------------------------------------
> 
> (Updated March 17, 2016, 7:07 p.m.)
> 
> 
> Review request for Ambari, Jonathan Hurley and Nate Cole.
> 
> 
> Bugs: AMBARI-15446
>     https://issues.apache.org/jira/browse/AMBARI-15446
> 
> 
> Repository: ambari
> 
> 
> Description
> -------
> 
> When a failure occurs during RU/EU and the task transitions to HOLDING_FAILED or HOLDING_TIMEDOUT, want Ambari to automatically retry up to up to x mins. This is useful when a host goes down as Ambari is running a task on it.
> ambari.properties will have 1 new parameter. E.g,. 
> stack-upgrade.max_retry_timeout_mins=15 (by default, will not be present)
> If Ambari Server is restarted, it should be able to recover.
> Today, Action Scheduler increases the attempt_count whenever a task is retried, but it requires resetting the start_time to -1. Because of this, we cannot rely on the start_time property to know when to timeout after several retries.
> 
> For the implementation, will add another thread to Ambari that will monitor failed tasks only during active RU/EU and change the status back to PENDING so that Action Scheduler can reschedule it.
> Luckily, any tasks in HOLDING_TIMEDOUT and HOLDING_FAILED states are blocking, so no other stages are allowed to proceed.
> In order to know when a task was first started, will add a new property to host_role_command table called original_start_time.
> 
> For the agents, we need to ensure that they always write out a response. On the first heartbeat, it should send the status of its last command so we know it failed and Ambari can retry.
> 
> 
> Diffs
> -----
> 
>   ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionDBAccessorImpl.java 429f573 
>   ambari-server/src/main/java/org/apache/ambari/server/actionmanager/HostRoleCommand.java 2764b3f 
>   ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java 3a80803 
>   ambari-server/src/main/java/org/apache/ambari/server/agent/HeartbeatProcessor.java a1a686a 
>   ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java PRE-CREATION 
>   ambari-server/src/main/java/org/apache/ambari/server/configuration/Configuration.java 9404506 
>   ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostRoleCommandDAO.java f5b1cb4 
>   ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostRoleCommandEntity.java 19f0602 
>   ambari-server/src/main/java/org/apache/ambari/server/topology/HostRequest.java 9eb514a 
>   ambari-server/src/main/java/org/apache/ambari/server/topology/LogicalRequest.java 82edbcf 
>   ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog240.java a803f73 
>   ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql 9b4810c 
>   ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql cc3d197 
>   ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql 07c786d 
>   ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql ab6dc93 
>   ambari-server/src/main/resources/Ambari-DDL-SQLAnywhere-CREATE.sql 8e91fde 
>   ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql 440ca44 
> 
> Diff: https://reviews.apache.org/r/44926/diff/
> 
> 
> Testing
> -------
> 
> Verified on a live cluster.
> 
> TODO: Still need to make more changes to the implementation, add the config, switch to gauva service, add a column, and add unit tests.
> 
> 
> Thanks,
> 
> Alejandro Fernandez
> 
>


Re: Review Request 44926: [DRAFT] Auto-retry on failure during RU/EU

Posted by Alejandro Fernandez <af...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/44926/
-----------------------------------------------------------

(Updated March 17, 2016, 11:07 p.m.)


Review request for Ambari, Jonathan Hurley and Nate Cole.


Changes
-------

Added more fixes, still need to write unit tests.


Bugs: AMBARI-15446
    https://issues.apache.org/jira/browse/AMBARI-15446


Repository: ambari


Description
-------

When a failure occurs during RU/EU and the task transitions to HOLDING_FAILED or HOLDING_TIMEDOUT, want Ambari to automatically retry up to up to x mins. This is useful when a host goes down as Ambari is running a task on it.
ambari.properties will have 1 new parameter. E.g,. 
stack-upgrade.max_retry_timeout_mins=15 (by default, will not be present)
If Ambari Server is restarted, it should be able to recover.
Today, Action Scheduler increases the attempt_count whenever a task is retried, but it requires resetting the start_time to -1. Because of this, we cannot rely on the start_time property to know when to timeout after several retries.

For the implementation, will add another thread to Ambari that will monitor failed tasks only during active RU/EU and change the status back to PENDING so that Action Scheduler can reschedule it.
Luckily, any tasks in HOLDING_TIMEDOUT and HOLDING_FAILED states are blocking, so no other stages are allowed to proceed.
In order to know when a task was first started, will add a new property to host_role_command table called original_start_time.

For the agents, we need to ensure that they always write out a response. On the first heartbeat, it should send the status of its last command so we know it failed and Ambari can retry.


Diffs (updated)
-----

  ambari-server/src/main/java/org/apache/ambari/server/actionmanager/ActionDBAccessorImpl.java 429f573 
  ambari-server/src/main/java/org/apache/ambari/server/actionmanager/HostRoleCommand.java 2764b3f 
  ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java 3a80803 
  ambari-server/src/main/java/org/apache/ambari/server/agent/HeartbeatProcessor.java a1a686a 
  ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java PRE-CREATION 
  ambari-server/src/main/java/org/apache/ambari/server/configuration/Configuration.java 9404506 
  ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostRoleCommandDAO.java f5b1cb4 
  ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostRoleCommandEntity.java 19f0602 
  ambari-server/src/main/java/org/apache/ambari/server/topology/HostRequest.java 9eb514a 
  ambari-server/src/main/java/org/apache/ambari/server/topology/LogicalRequest.java 82edbcf 
  ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog240.java a803f73 
  ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql 9b4810c 
  ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql cc3d197 
  ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql 07c786d 
  ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql ab6dc93 
  ambari-server/src/main/resources/Ambari-DDL-SQLAnywhere-CREATE.sql 8e91fde 
  ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql 440ca44 

Diff: https://reviews.apache.org/r/44926/diff/


Testing
-------

Verified on a live cluster.

TODO: Still need to make more changes to the implementation, add the config, switch to gauva service, add a column, and add unit tests.


Thanks,

Alejandro Fernandez


Re: Review Request 44926: [DRAFT] Auto-retry on failure during RU/EU

Posted by Alejandro Fernandez <af...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/44926/#review123924
-----------------------------------------------------------




ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java (line 178)
<https://reviews.apache.org/r/44926/#comment186253>

    Change to guava service



ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (line 233)
<https://reviews.apache.org/r/44926/#comment186252>

    Will change to query from cache instead of DB



ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (line 290)
<https://reviews.apache.org/r/44926/#comment186251>

    On Ambari Server restart, this may keep scheduling more than wanted. So will add a column called original_start_time to host_role_command.


- Alejandro Fernandez


On March 16, 2016, 8:41 p.m., Alejandro Fernandez wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/44926/
> -----------------------------------------------------------
> 
> (Updated March 16, 2016, 8:41 p.m.)
> 
> 
> Review request for Ambari, Jonathan Hurley and Nate Cole.
> 
> 
> Bugs: AMBARI-15446
>     https://issues.apache.org/jira/browse/AMBARI-15446
> 
> 
> Repository: ambari
> 
> 
> Description
> -------
> 
> When a failure occurs during RU/EU and the task transitions to HOLDING_FAILED or HOLDING_TIMEDOUT, want Ambari to automatically retry up to up to x mins. This is useful when a host goes down as Ambari is running a task on it.
> ambari.properties will have 1 new parameter. E.g,. 
> stack-upgrade.max_retry_timeout_mins=15 (by default, will not be present)
> If Ambari Server is restarted, it should be able to recover.
> Today, Action Scheduler increases the attempt_count whenever a task is retried, but it requires resetting the start_time to -1. Because of this, we cannot rely on the start_time property to know when to timeout after several retries.
> 
> For the implementation, will add another thread to Ambari that will monitor failed tasks only during active RU/EU and change the status back to PENDING so that Action Scheduler can reschedule it.
> Luckily, any tasks in HOLDING_TIMEDOUT and HOLDING_FAILED states are blocking, so no other stages are allowed to proceed.
> In order to know when a task was first started, will add a new property to host_role_command table called original_start_time.
> 
> For the agents, we need to ensure that they always write out a response. On the first heartbeat, it should send the status of its last command so we know it failed and Ambari can retry.
> 
> 
> Diffs
> -----
> 
>   ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java 3a80803 
>   ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java PRE-CREATION 
>   ambari-server/src/main/java/org/apache/ambari/server/checks/PreviousUpgradeCompleted.java 3a4467f 
>   ambari-server/src/main/java/org/apache/ambari/server/orm/dao/ClusterVersionDAO.java 1bcca60 
>   ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostRoleCommandDAO.java f5b1cb4 
>   ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ClusterVersionEntity.java f1867b4 
>   ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostRoleCommandEntity.java 19f0602 
>   ambari-server/src/main/java/org/apache/ambari/server/state/Cluster.java ed3c772 
>   ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClusterImpl.java 1c7ff61 
>   ambari-server/src/main/java/org/apache/ambari/server/topology/LogicalRequest.java 82edbcf 
> 
> Diff: https://reviews.apache.org/r/44926/diff/
> 
> 
> Testing
> -------
> 
> Verified on a live cluster.
> 
> TODO: Still need to make more changes to the implementation, add the config, switch to gauva service, add a column, and add unit tests.
> 
> 
> Thanks,
> 
> Alejandro Fernandez
> 
>


Re: Review Request 44926: [DRAFT] Auto-retry on failure during RU/EU

Posted by Nate Cole <nc...@hortonworks.com>.
-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/44926/#review123927
-----------------------------------------------------------




ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (lines 255 - 262)
<https://reviews.apache.org/r/44926/#comment186257>

    Can be only one, even if it's a downgrade.



ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java (line 277)
<https://reviews.apache.org/r/44926/#comment186258>

    I think you know why this and others like it can't go into a commit :)


- Nate Cole


On March 16, 2016, 4:41 p.m., Alejandro Fernandez wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/44926/
> -----------------------------------------------------------
> 
> (Updated March 16, 2016, 4:41 p.m.)
> 
> 
> Review request for Ambari, Jonathan Hurley and Nate Cole.
> 
> 
> Bugs: AMBARI-15446
>     https://issues.apache.org/jira/browse/AMBARI-15446
> 
> 
> Repository: ambari
> 
> 
> Description
> -------
> 
> When a failure occurs during RU/EU and the task transitions to HOLDING_FAILED or HOLDING_TIMEDOUT, want Ambari to automatically retry up to up to x mins. This is useful when a host goes down as Ambari is running a task on it.
> ambari.properties will have 1 new parameter. E.g,. 
> stack-upgrade.max_retry_timeout_mins=15 (by default, will not be present)
> If Ambari Server is restarted, it should be able to recover.
> Today, Action Scheduler increases the attempt_count whenever a task is retried, but it requires resetting the start_time to -1. Because of this, we cannot rely on the start_time property to know when to timeout after several retries.
> 
> For the implementation, will add another thread to Ambari that will monitor failed tasks only during active RU/EU and change the status back to PENDING so that Action Scheduler can reschedule it.
> Luckily, any tasks in HOLDING_TIMEDOUT and HOLDING_FAILED states are blocking, so no other stages are allowed to proceed.
> In order to know when a task was first started, will add a new property to host_role_command table called original_start_time.
> 
> For the agents, we need to ensure that they always write out a response. On the first heartbeat, it should send the status of its last command so we know it failed and Ambari can retry.
> 
> 
> Diffs
> -----
> 
>   ambari-server/src/main/java/org/apache/ambari/server/agent/HeartBeatHandler.java 3a80803 
>   ambari-server/src/main/java/org/apache/ambari/server/agent/RetryActionMonitor.java PRE-CREATION 
>   ambari-server/src/main/java/org/apache/ambari/server/checks/PreviousUpgradeCompleted.java 3a4467f 
>   ambari-server/src/main/java/org/apache/ambari/server/orm/dao/ClusterVersionDAO.java 1bcca60 
>   ambari-server/src/main/java/org/apache/ambari/server/orm/dao/HostRoleCommandDAO.java f5b1cb4 
>   ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ClusterVersionEntity.java f1867b4 
>   ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostRoleCommandEntity.java 19f0602 
>   ambari-server/src/main/java/org/apache/ambari/server/state/Cluster.java ed3c772 
>   ambari-server/src/main/java/org/apache/ambari/server/state/cluster/ClusterImpl.java 1c7ff61 
>   ambari-server/src/main/java/org/apache/ambari/server/topology/LogicalRequest.java 82edbcf 
> 
> Diff: https://reviews.apache.org/r/44926/diff/
> 
> 
> Testing
> -------
> 
> Verified on a live cluster.
> 
> TODO: Still need to make more changes to the implementation, add the config, switch to gauva service, add a column, and add unit tests.
> 
> 
> Thanks,
> 
> Alejandro Fernandez
> 
>