Details
Description
This came up during testing the ITBLL on a custom cluster.
Master aborts if WAL splitting fails:
2016-08-03 19:56:57,693 INFO [B.priority.fifo.QRpcServer.handler=4,queue=0,port=20000] master.ServerManager: Registering server=hbase-bug58335-6.openstacklocal,16020,1470254212713 2016-08-03 19:57:00,062 INFO [main-EventThread] coordination.SplitLogManagerCoordination: task /hbase-secure/splitWAL/WALs%2Fhbase-bug58335-6.openstacklocal%2C16020%2C1470253655579-splitting%2Fhbase-bug58335-6.openstacklocal%252C16020%252C1470253655579.default.1470253660327 entered state: ERR hbase-bug58335-4.openstacklocal,16020,1470253592920 2016-08-03 19:57:00,064 WARN [main-EventThread] coordination.SplitLogManagerCoordination: Error splitting /hbase-secure/splitWAL/WALs%2Fhbase-bug58335-6.openstacklocal%2C16020%2C1470253655579-splitting%2Fhbase-bug58335-6.openstacklocal%252C16020%252C1470253655579.default.1470253660327 2016-08-03 19:57:00,064 WARN [MASTER_SERVER_OPERATIONS-hbase-bug58335-7:20000-0] master.SplitLogManager: error while splitting logs in [hdfs://hbase-bug58335-7.openstacklocal:8020/apps/hbase/data/WALs/hbase-bug58335-6.openstacklocal,16020,1470253655579-splitting] installed = 1 but only 0 done 2016-08-03 19:57:00,065 ERROR [MASTER_SERVER_OPERATIONS-hbase-bug58335-7:20000-0] executor.EventHandler: Caught throwable while processing event M_SERVER_SHUTDOWN java.io.IOException: failed log splitting for hbase-bug58335-6.openstacklocal,16020,1470253655579, will retry at org.apache.hadoop.hbase.master.handler.ServerShutdownHandler.resubmit(ServerShutdownHandler.java:357) at org.apache.hadoop.hbase.master.handler.ServerShutdownHandler.process(ServerShutdownHandler.java:220) at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:129) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) Caused by: java.io.IOException: error or interrupted while splitting logs in [hdfs://hbase-bug58335-7.openstacklocal:8020/apps/hbase/data/WALs/hbase-bug58335-6.openstacklocal,16020,1470253655579-splitting] Task = installed = 1 done = 0 error = 1 at org.apache.hadoop.hbase.master.SplitLogManager.splitLogDistributed(SplitLogManager.java:290) at org.apache.hadoop.hbase.master.MasterFileSystem.splitLog(MasterFileSystem.java:393) at org.apache.hadoop.hbase.master.MasterFileSystem.splitLog(MasterFileSystem.java:366) at org.apache.hadoop.hbase.master.MasterFileSystem.splitLog(MasterFileSystem.java:288) at org.apache.hadoop.hbase.master.handler.ServerShutdownHandler.process(ServerShutdownHandler.java:213) ... 4 more 2016-08-03 19:57:00,067 FATAL [MASTER_SERVER_OPERATIONS-hbase-bug58335-7:20000-0] master.HMaster: Master server abort: loaded coprocessors are: [org.apache.ranger.authorization.hbase.RangerAuthorizationCoprocessor, org.apache.hadoop.hbase.backup.master.BackupController] 2016-08-03 19:57:00,067 FATAL [MASTER_SERVER_OPERATIONS-hbase-bug58335-7:20000-0] master.HMaster: Caught throwable while processing event M_SERVER_SHUTDOWN java.io.IOException: failed log splitting for hbase-bug58335-6.openstacklocal,16020,1470253655579, will retry at org.apache.hadoop.hbase.master.handler.ServerShutdownHandler.resubmit(ServerShutdownHandler.java:357) at org.apache.hadoop.hbase.master.handler.ServerShutdownHandler.process(ServerShutdownHandler.java:220) at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:129) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) Caused by: java.io.IOException: error or interrupted while splitting logs in [hdfs://hbase-bug58335-7.openstacklocal:8020/apps/hbase/data/WALs/hbase-bug58335-6.openstacklocal,16020,1470253655579-splitting] Task = installed = 1 done = 0 error = 1 at org.apache.hadoop.hbase.master.SplitLogManager.splitLogDistributed(SplitLogManager.java:290) at org.apache.hadoop.hbase.master.MasterFileSystem.splitLog(MasterFileSystem.java:393) at org.apache.hadoop.hbase.master.MasterFileSystem.splitLog(MasterFileSystem.java:366) at org.apache.hadoop.hbase.master.MasterFileSystem.splitLog(MasterFileSystem.java:288) at org.apache.hadoop.hbase.master.handler.ServerShutdownHandler.process(ServerShutdownHandler.java:213) ... 4 more
If we fail to split the WAL, we normally retry, but it seems that it is doing this by throwing an exception:
private void resubmit(final ServerName serverName, IOException ex) throws IOException { // typecast to SSH so that we make sure that it is the SSH instance that // gets submitted as opposed to MSSH or some other derived instance of SSH this.services.getExecutorService().submit((ServerShutdownHandler) this); this.deadServers.add(serverName); throw new IOException("failed log splitting for " + serverName + ", will retry", ex); }
HBASE-14968 made the change that if the handler is throwing an uncaught exception, we abort the server:
protected void handleException(Throwable t) { String msg = "Caught throwable while processing event " + eventType; LOG.error(msg, t); if (server != null) { server.abort(msg, t); } }
It seems that SSH, and some other handlers are throwing exceptions in valid cases to retry the operations. We can undo the server.abort() call which was originally added in HBASE-14968 for cases where unchecked exceptions like ConcurrentModificationException, etc are not ignored.
Attachments
Attachments
Issue Links
- is related to
-
HBASE-14968 ConcurrentModificationException in region close resulting in the region staying in closing state
- Closed