|
@@ -42,6 +42,7 @@ import org.apache.hadoop.yarn.api.records.Container;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
|
|
|
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
@@ -820,14 +821,18 @@ public abstract class RMStateStore extends AbstractService {
|
|
|
* @param failureCause the exception due to which the operation failed
|
|
|
*/
|
|
|
protected void notifyStoreOperationFailed(Exception failureCause) {
|
|
|
+ LOG.error("State store operation failed ", failureCause);
|
|
|
if (failureCause instanceof StoreFencedException) {
|
|
|
Thread standByTransitionThread =
|
|
|
new Thread(new StandByTransitionThread());
|
|
|
standByTransitionThread.setName("StandByTransitionThread Handler");
|
|
|
standByTransitionThread.start();
|
|
|
} else {
|
|
|
- rmDispatcher.getEventHandler().handle(
|
|
|
- new RMFatalEvent(RMFatalEventType.STATE_STORE_OP_FAILED, failureCause));
|
|
|
+ if (YarnConfiguration.shouldRMFailFast(getConfig())) {
|
|
|
+ rmDispatcher.getEventHandler().handle(
|
|
|
+ new RMFatalEvent(RMFatalEventType.STATE_STORE_OP_FAILED,
|
|
|
+ failureCause));
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|