HDFS-12683. DFSZKFailOverController re-order logic for logging Exception. Contributed by Bharat Viswanadham.

This commit is contained in:
Arpit Agarwal 2017-10-23 10:47:16 -07:00
parent ef0d6ff2e6
commit a373094bf9
2 changed files with 30 additions and 21 deletions

View File

@ -183,23 +183,23 @@ public abstract class ZKFailoverController {
} }
}); });
} catch (RuntimeException rte) { } catch (RuntimeException rte) {
LOG.error("The failover controller encounters runtime error: " + rte);
throw (Exception)rte.getCause(); throw (Exception)rte.getCause();
} }
} }
private int doRun(String[] args) private int doRun(String[] args)
throws HadoopIllegalArgumentException, IOException, InterruptedException { throws Exception {
try { try {
initZK(); initZK();
} catch (KeeperException ke) { } catch (KeeperException ke) {
LOG.error("Unable to start failover controller. Unable to connect " LOG.error("Unable to start failover controller. Unable to connect "
+ "to ZooKeeper quorum at " + zkQuorum + ". Please check the " + "to ZooKeeper quorum at " + zkQuorum + ". Please check the "
+ "configured value for " + ZK_QUORUM_KEY + " and ensure that " + "configured value for " + ZK_QUORUM_KEY + " and ensure that "
+ "ZooKeeper is running."); + "ZooKeeper is running.", ke);
return ERR_CODE_NO_ZK; return ERR_CODE_NO_ZK;
} }
try {
if (args.length > 0) { if (args.length > 0) {
if ("-formatZK".equals(args[0])) { if ("-formatZK".equals(args[0])) {
boolean force = false; boolean force = false;
@ -214,10 +214,15 @@ public abstract class ZKFailoverController {
} }
} }
return formatZK(force, interactive); return formatZK(force, interactive);
} else { }
else {
badArg(args[0]); badArg(args[0]);
} }
} }
} catch (Exception e){
LOG.error("The failover controller encounters runtime error", e);
throw e;
}
if (!elector.parentZNodeExists()) { if (!elector.parentZNodeExists()) {
LOG.error("Unable to start failover controller. " LOG.error("Unable to start failover controller. "
@ -235,11 +240,14 @@ public abstract class ZKFailoverController {
return ERR_CODE_NO_FENCER; return ERR_CODE_NO_FENCER;
} }
try {
initRPC(); initRPC();
initHM(); initHM();
startRPC(); startRPC();
try {
mainLoop(); mainLoop();
} catch (Exception e) {
LOG.error("The failover controller encounters runtime error: ", e);
throw e;
} finally { } finally {
rpcServer.stopAndJoin(); rpcServer.stopAndJoin();

View File

@ -192,7 +192,8 @@ public class DFSZKFailoverController extends ZKFailoverController {
try { try {
retCode = zkfc.run(parser.getRemainingArgs()); retCode = zkfc.run(parser.getRemainingArgs());
} catch (Throwable t) { } catch (Throwable t) {
LOG.fatal("Got a fatal error, exiting now", t); LOG.fatal("DFSZKFailOverController exiting due to earlier exception "
+ t);
} }
System.exit(retCode); System.exit(retCode);
} }