[
https://issues.apache.org/jira/browse/HBASE-13217?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14527681#comment-14527681
]
Matteo Bertozzi commented on HBASE-13217:
-----------------------------------------
I haven't looked at the code in awhile but in theory, the idea was:
* master broadcast the new operation via zk
* each RS read the request, and it executes only if the request can be applied
to the RS. (e.g. flush(table=foo) if that RS has no "foo" table has nothing to
do, keep in mind that the master in general snapshot/flush keeps a list of
"regions online" so the master already assume that kind of behavior from the RS)
proc-v2 (not part of this jira) has a different way of sending messages to the
regions involved in this operation and is based on AM notifications since we
will tolerate move, but the idea is like the above. if the RS has nothing todo
with what we asked it will not partecipate.
> Flush procedure fails in trunk due to ZK issue
> ----------------------------------------------
>
> Key: HBASE-13217
> URL: https://issues.apache.org/jira/browse/HBASE-13217
> Project: HBase
> Issue Type: Bug
> Reporter: ramkrishna.s.vasudevan
> Assignee: Stephen Yuan Jiang
>
> When ever I try to flush explicitly in the trunk code the flush procedure
> fails due to ZK issue
> {code}
> ERROR: org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable
> via
> stobdtserver3,16040,1426172670959:org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable:
> java.io.IOException: org.apache.zookeeper.KeeperException$NoNodeException:
> KeeperErrorCode = NoNode for
> /hbase/flush-table-proc/acquired/TestTable/stobdtserver3,16040,1426172670959
> at
> org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher.rethrowException(ForeignExceptionDispatcher.java:83)
> at
> org.apache.hadoop.hbase.procedure.Procedure.isCompleted(Procedure.java:368)
> at
> org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager.isProcedureDone(MasterFlushTableProcedureManager.java:196)
> at
> org.apache.hadoop.hbase.master.MasterRpcServices.isProcedureDone(MasterRpcServices.java:905)
> at
> org.apache.hadoop.hbase.protobuf.generated.MasterProtos$MasterService$2.callBlockingMethod(MasterProtos.java:47019)
> at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2073)
> at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:107)
> 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)
> Caused by:
> org.apache.hadoop.hbase.errorhandling.ForeignException$ProxyThrowable:
> java.io.IOException: org.apache.zookeeper.KeeperException$NoNodeException:
> KeeperErrorCode = NoNode for
> /hbase/flush-table-proc/acquired/TestTable/stobdtserver3,16040,1426172670959
> at
> org.apache.hadoop.hbase.procedure.Subprocedure.cancel(Subprocedure.java:273)
> at
> org.apache.hadoop.hbase.procedure.ProcedureMember.controllerConnectionFailure(ProcedureMember.java:225)
> at
> org.apache.hadoop.hbase.procedure.ZKProcedureMemberRpcs.sendMemberAcquired(ZKProcedureMemberRpcs.java:254)
> at
> org.apache.hadoop.hbase.procedure.Subprocedure.call(Subprocedure.java:166)
> at
> org.apache.hadoop.hbase.procedure.Subprocedure.call(Subprocedure.java:52)
> at java.util.concurrent.FutureTask.run(FutureTask.java:262)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> ... 1 more
> {code}
> Once this occurs, even on restart of the RS the RS becomes unusable. I have
> verified that the ZK remains intact and there is no problem with it. a bit
> older version of trunk ( 3months) does not have this problem.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)