Xiaolin Ha created HBASE-22940:
----------------------------------

             Summary: Snapshot NoNode error
                 Key: HBASE-22940
                 URL: https://issues.apache.org/jira/browse/HBASE-22940
             Project: HBase
          Issue Type: Bug
          Components: snapshots
            Reporter: Xiaolin Ha
            Assignee: Xiaolin Ha


When we take snapshot for thousands tables on our cluster, we found there 
occasionally occurs NoNodeException,error stack is as follows,
{quote}ERROR: org.apache.hadoop.hbase.snapshot.HBaseSnapshotException: Snapshot 
\{ ss=KYLIN_2JAU7T91XU_mtzjyprc 
table=kylin_zjyprc_bigdata_staging:KYLIN_2JAU7T91XU type=FLUSH } had an error. 
Procedure KYLIN_2JAU7T91XU_mtzjyprc \{ waiting=[] done=[] } at 
org.apache.hadoop.hbase.master.snapshot.SnapshotManager.isSnapshotDone(SnapshotManager.java:350)
 at org.apache.hadoop.hbase.master.HMaster.isSnapshotDone(HMaster.java:3674) at 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos$MasterService$2.callBlockingMethod(MasterProtos.java:44817)
 at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2059) at 
org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:126) at 
org.apache.hadoop.hbase.ipc.MasterFifoRpcScheduler.lambda$dispatch$1(MasterFifoRpcScheduler.java:68)
 at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at 
java.util.concurrent.FutureTask.run(FutureTask.java:266) at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 
at java.lang.Thread.run(Thread.java:748) Caused by: 
org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable via 
zjy-hadoop-prc-st1309.bj,24600,1557969473924:org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable:
 java.io.IOException: org.apache.zookeeper.KeeperException$NoNodeException: 
KeeperErrorCode = NoNode for 
/hbase/zjyprc-xiaomi/online-snapshot/reached/KYLIN_2JAU7T91XU_mtzjyprc/zjy-hadoop-prc-st1309.bj,24600,1557969473924
 at 
org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher.rethrowException(ForeignExceptionDispatcher.java:83)
 at 
org.apache.hadoop.hbase.master.snapshot.TakeSnapshotHandler.rethrowExceptionIfFailed(TakeSnapshotHandler.java:312)
 at 
org.apache.hadoop.hbase.master.snapshot.SnapshotManager.isSnapshotDone(SnapshotManager.java:340)
 ... 10 more Caused by: 
org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable: 
java.io.IOException: org.apache.zookeeper.KeeperException$NoNodeException: 
KeeperErrorCode = NoNode for 
/hbase/zjyprc-xiaomi/online-snapshot/reached/KYLIN_2JAU7T91XU_mtzjyprc/zjy-hadoop-prc-st1309.bj,24600,1557969473924
 at 
org.apache.hadoop.hbase.procedure.Subprocedure.cancel(Subprocedure.java:270) at 
org.apache.hadoop.hbase.procedure.ProcedureMember.controllerConnectionFailure(ProcedureMember.java:225)
 at 
org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs.sendMemberCompleted(ZKProcedureMemberRpcs.java:267)
 at org.apache.hadoop.hbase.procedure.Subprocedure.call(Subprocedure.java:185) 
at org.apache.hadoop.hbase.procedure.Subprocedure.call(Subprocedure.java:52) 
... 4 more @zjy-hadoop-prc-zk05.bj/10.152.48.41:24500 Here is some help for 
this command: Take a snapshot of specified table. Examples: hbase> snapshot 
'sourceTable', 'snapshotName' hbase> snapshot 'namespace:sourceTable', 
'snapshotName', \{SKIP_FLUSH => true}
{quote}
I looked through relevant server logs, and found that currently implementation 
of snapshot has some problems.  When creating Procedure for snapshot, the 
regions servers where table regions on will be set as  acquired and released 
barriers. Master watches zk and if all the barrier region servers have added 
nodes to the parent reached node, coordinator releases ALL the barriers and 
snapshot procedure will be thought as completed. Followed by the relevant 
parent reached/required node be cleared by `resetMembers()`. But all the region 
servers will add node to the parent reached/required node, so non-barrier 
region servers add children will encounter NoNodeException at this time.

We think the coordinator only set relevant region servers as barriers may be 
not enough. All region servers adds node and may be all can be barriers.

 

 

 



--
This message was sent by Atlassian Jira
(v8.3.2#803003)

Reply via email to