hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "stack (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-13567) [DLR] Region stuck in recovering mode
Date Thu, 30 Apr 2015 06:18:08 GMT

    [ https://issues.apache.org/jira/browse/HBASE-13567?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14520940#comment-14520940
] 

stack commented on HBASE-13567:
-------------------------------

Should have known better... Our [~jeffreyz] added code to the master startup to purge recovering
zk nodes if old logs in the fs yet to be split. It is being frustrated though by the fact
that SplitLogManagerCoordination is started long before we get to the finishActiveMasterInitialization
and it is doing cleanup and archiving of logs before this purge recovering code can run. This
purge code might also trigger while replaying is still ongoing. Let me try and fix.

Meantime, here is the evidence:

# Master starts up:

Fri Apr 24 22:40:42 PDT 2015 Starting master on c2020.halxg.cloudera.com

# SLMC is started in superclass regionserver constructor:

2015-04-24 22:40:48,221 DEBUG [c2020:16000.activeMasterManager] coordination.SplitLogManagerCoordination:
Distributed log replay=true

# The last file to be split on the dead server has been processed while the master was done.
The Master finds the orphan task in SLMC#init:

2015-04-24 22:40:48,229 INFO  [c2020:16000.activeMasterManager] coordination.SplitLogManagerCoordination:
found orphan task WALs%2Fc2023.halxg.cloudera.com%2C16020%2C1429939702612-splitting%2Fc2023.halxg.cloudera.com%252C16020%252C1429939702612.default.1429940160038

# It notices it done:

2015-04-24 22:40:48,263 INFO  [main-EventThread] coordination.SplitLogManagerCoordination:
task /hbase/splitWAL/WALs%2Fc2023.halxg.cloudera.com%2C16020%2C1429939702612-splitting%2Fc2023.halxg.cloudera.com%252C16020%252C1429939702612.default.1429940160038
entered state: DONE c2021.halxg.cloudera.com,16020,1429940428614

# ... then archives it:

2015-04-24 22:40:48,306 INFO  [main-EventThread] wal.WALSplitter: Archived processed log hdfs://c2020.halxg.cloudera.com:8020/hbase/WALs/c2023.halxg.cloudera.com,16020,1429939702612-splitting/c2023.halxg.cloudera.com%2C16020%2C1429939702612.default.1429940160038
to hdfs://c2020.halxg.cloudera.com:8020/hbase/oldWALs/c2023.halxg.cloudera.com%2C16020%2C142993970261

# Only much later does the review of fs happen...

2015-04-24 22:40:57,330 INFO  [c2020:16000.activeMasterManager] master.MasterFileSystem: Log
folder hdfs://c2020.halxg.cloudera.com:8020/hbase/WALs/c2021.halxg.cloudera.com,16020,1429940121416-splitting
doesn't belong to a known region server, splitting...

# And so on...




> [DLR] Region stuck in recovering mode
> -------------------------------------
>
>                 Key: HBASE-13567
>                 URL: https://issues.apache.org/jira/browse/HBASE-13567
>             Project: HBase
>          Issue Type: Bug
>          Components: wal
>    Affects Versions: 1.1.0
>            Reporter: stack
>            Assignee: stack
>
> On the hosting server, it says:
> {code}
> 2015-04-24 22:47:27,736 DEBUG [PriorityRpcServer.handler=2,queue=0,port=16020] ipc.RpcServer:
PriorityRpcServer.handler=2,queue=0,port=16020: callId: 84 service: ClientService methodName:
Get size: 99 connection: 10.20.84.26:52860
> org.apache.hadoop.hbase.exceptions.RegionInRecoveryException: hbase:namespace,,1425361558502.57b3dc571e5753b306509b5740cd25b9.
is recovering; cannot take reads
>         at org.apache.hadoop.hbase.regionserver.HRegion.startRegionOperation(HRegion.java:7530)
>         at org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:2427)
>         at org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:2422)
>         at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:6451)
>         at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:6430)
>         at org.apache.hadoop.hbase.regionserver.RSRpcServices.get(RSRpcServices.java:1898)
>         at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32201)
>         at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2112)
>         at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:101)
>         at org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:130)
>         at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:107)
>         at java.lang.Thread.run(Thread.java:745)
> {code}
> whenever someone tries to read the region. In the above case, it is the master trying
to initialize after being killed by a monkey. It is trying to set up the TableNamespaceManager.
Eventually it fails after 350 attempts:
> {code}
> 2015-04-25 00:35:58,750 WARN  [c2020:16000.activeMasterManager] master.TableNamespaceManager:
Caught exception in initializing namespace table manager
> 193959 org.apache.hadoop.hbase.client.RetriesExhaustedException: Failed after attempts=350,
exceptions:
> 193960 Fri Apr 24 22:40:57 PDT 2015, RpcRetryingCaller{globalStartTime=1429940457781,
pause=100, retries=350}, org.apache.hadoop.hbase.exceptions.RegionInRecoveryException: org.apache.hadoop.hbase.excepti
      ons.RegionInRecoveryException: hbase:namespace,,1425361558502.57b3dc571e5753b306509b5740cd25b9.
is recovering; cannot take reads
> 193961         at org.apache.hadoop.hbase.regionserver.HRegion.startRegionOperation(HRegion.java:7530)
> 193962         at org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:2427)
> 193963         at org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:2422)
> 193964         at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:6451)
> 193965         at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:6430)
> 193966         at org.apache.hadoop.hbase.regionserver.RSRpcServices.get(RSRpcServices.java:1898)
> 193967         at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32201)
> {code}
> The master is supposed to have 'processed' the region -- the hbase:namespace in this
case -- but the regionserver did not get the notification:
> 184849 2015-04-24 22:46:31,650 DEBUG [M_LOG_REPLAY_OPS-c2020:16000-8] coordination.SplitLogManagerCoordination:
Processing recovering [3fbee1781e0c2ded3cc30b701a03797d, 3f4ea5ea14653cee6006f13c7d06d10b,
e       52b81449d08921c49625620cfc7ace7, 07459e75bef40ec82b6d4267c9de9971, b26731667a5e0f15162ad4fa3408b99c,
57b3dc571e5753b306509b5740cd25b9, 349eadd360d57083a88e0d84bcb29351, 99e3eb2bcf44bccded24103e351c
      96b6] and servers [c2021.halxg.cloudera.com,16020,1429940280208], isMetaRecovery=false
> Do we need to keep sending notification until acknowledged by the regionserver as we
do w/ split?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message