diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java index 7ada04a6ac9..986b42cb412 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java @@ -183,40 +183,45 @@ public abstract class ZKFailoverController { } }); } catch (RuntimeException rte) { - LOG.error("The failover controller encounters runtime error: " + rte); throw (Exception)rte.getCause(); } } private int doRun(String[] args) - throws HadoopIllegalArgumentException, IOException, InterruptedException { + throws Exception { try { initZK(); } catch (KeeperException ke) { LOG.error("Unable to start failover controller. Unable to connect " + "to ZooKeeper quorum at " + zkQuorum + ". Please check the " + "configured value for " + ZK_QUORUM_KEY + " and ensure that " - + "ZooKeeper is running."); + + "ZooKeeper is running.", ke); return ERR_CODE_NO_ZK; } - if (args.length > 0) { - if ("-formatZK".equals(args[0])) { - boolean force = false; - boolean interactive = true; - for (int i = 1; i < args.length; i++) { - if ("-force".equals(args[i])) { - force = true; - } else if ("-nonInteractive".equals(args[i])) { - interactive = false; - } else { - badArg(args[i]); + try { + if (args.length > 0) { + if ("-formatZK".equals(args[0])) { + boolean force = false; + boolean interactive = true; + for (int i = 1; i < args.length; i++) { + if ("-force".equals(args[i])) { + force = true; + } else if ("-nonInteractive".equals(args[i])) { + interactive = false; + } else { + badArg(args[i]); + } } + return formatZK(force, interactive); + } + else { + badArg(args[0]); } - return formatZK(force, interactive); - } else { - badArg(args[0]); } + } catch (Exception e){ + LOG.error("The failover controller encounters runtime error", e); + throw e; } if (!elector.parentZNodeExists()) { @@ -235,11 +240,14 @@ public abstract class ZKFailoverController { return ERR_CODE_NO_FENCER; } - initRPC(); - initHM(); - startRPC(); try { + initRPC(); + initHM(); + startRPC(); mainLoop(); + } catch (Exception e) { + LOG.error("The failover controller encounters runtime error: ", e); + throw e; } finally { rpcServer.stopAndJoin(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java index ff3fd68b738..a710748b014 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java @@ -192,7 +192,8 @@ public class DFSZKFailoverController extends ZKFailoverController { try { retCode = zkfc.run(parser.getRemainingArgs()); } catch (Throwable t) { - LOG.fatal("Got a fatal error, exiting now", t); + LOG.fatal("DFSZKFailOverController exiting due to earlier exception " + + t); } System.exit(retCode); }