Details
-
Bug
-
Status: Patch Available
-
Major
-
Resolution: Unresolved
-
2.9.2
-
Fix standby RM hangs (not retry or crash) forever due to forever lost from leader election. And now, RM will only transition to standby for known safe fatal events.
-
Patch, Important
Description
Issue Summary:
Standby RM hangs (not retry or crash) forever due to forever lost from leader election
Issue Repro Steps:
- Start multiple RMs in HA mode
- Modify all hostnames in the zk connect string to different values in DNS.
(In reality, we need to replace old/bad zk machines to new/good zk machines, so their DNS hostname will be changed.)
Issue Logs:
See the full RM log in attachment, yarn_rm.zip (The RM is BN4SCH101222318).
To make it clear, the whole story is:
Join Election Win the leader (ZK Node Creation Callback) Start to becomeActive Start RMActiveServices Start CommonNodeLabelsManager failed due to zk connect UnknownHostException Stop CommonNodeLabelsManager Stop RMActiveServices Create and Init RMActiveServices Fail to becomeActive ReJoin Election Failed to Join Election due to zk connect UnknownHostException (Here the exception is eat and just send event) Send EMBEDDED_ELECTOR_FAILED RMFatalEvent to transition RM to standby Transitioning RM to Standby Start StandByTransitionThread Already in standby state ReJoin Election Failed to Join Election due to zk connect UnknownHostException (Here the exception is eat and just send event) Send EMBEDDED_ELECTOR_FAILED RMFatalEvent to transition RM to standby Transitioning RM to Standby Start StandByTransitionThread Found RMActiveServices's StandByTransitionRunnable object has already run previously, so immediately return
The standby RM failed to rejoin the election, but it will never retry or crash later, so afterwards no zk related logs and the standby RM is forever hang, even if the zk connect string hostnames are changed back the orignal ones in DNS.
So, this should be a bug in RM, because RM should always try to join election (give up join election should only happen on RM decide to crash), otherwise, a RM without inside the election can never become active again and start real works.
Caused By:
It is introduced by YARN-3742
The JIRA want to improve is that, when STATE_STORE_OP_FAILED RMFatalEvent happens, RM should transition to standby, instead of crash.
However, in fact, the JIRA makes ALL kinds of RMFatalEvent ONLY transition to standby, instead of crash. (In contrast, before this change, RM makes all to crash instead of to standby)
So, even if EMBEDDED_ELECTOR_FAILED or CRITICAL_THREAD_CRASH happens, it will leave the standby RM continue not work, such as stay in standby forever.
And as the author said:
I think a good approach here would be to change the RMFatalEvent handler to transition to standby as the default reaction, with shutdown as a special case for certain types of failures.
But the author is too optimistic when implement the patch.
What the Patch's solution:
So, for conservative, we would better only transition to standby for the failures in whitelist:
public enum RMFatalEventType {
// Source <- Store
STATE_STORE_FENCED,
STATE_STORE_OP_FAILED,
// Source <- Embedded Elector
EMBEDDED_ELECTOR_FAILED,
// Source <- Admin Service
TRANSITION_TO_ACTIVE_FAILED,
// Source <- Critical Thread Crash
CRITICAL_THREAD_CRASH
}
And others, such as EMBEDDED_ELECTOR_FAILED or CRITICAL_THREAD_CRASH and future added failure types (until we triaged it to be in whitelist), should crash RM, because we cannot ensure that they will never cause RM cannot work in standby state, and the conservative way is to crash RM.
Besides, after crash, the RM's external watchdog service can know this and try to repair the RM machine, send alerts, etc.
And the RM can reload the latest zk connect string config with the latest hostnames.
For more details, please check the patch.
Attachments
Attachments
Issue Links
- is caused by
-
YARN-3742 YARN RM will shut down if ZKClient creation times out
- Resolved
- is related to
-
ZOOKEEPER-1576 Zookeeper cluster - failed to connect to cluster if one of the provided IPs causes java.net.UnknownHostException
- Resolved
-
YARN-4438 Implement RM leader election with curator
- Resolved
-
YARN-6061 Add an UncaughtExceptionHandler for critical threads in RM
- Resolved