[
https://issues.apache.org/jira/browse/HBASE-13217?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14526238#comment-14526238
]
ramkrishna.s.vasudevan commented on HBASE-13217:
------------------------------------------------
bbq.ramkrishna.s.vasudevan are you still seeing this issue? I am curious about
your investigation and why you close the issue (cannot repro from master?)
ACtually this thing happened to me continuously when I upgraded my internal
version to one of the master version during that time. whenever I called flush
this was happening. I did not get in to the details of it due to lack of time.
Then later i upgraded the version that I had and this problem went off. So one
reason for closing was if there was some problem in my internal version this
could have happened, but one thing to note is that that had nothing to do with
flushes and procedures. But had a doubt that if there was any exception
(runtime) then the flush procedures where not able to identify that (Just a
wild guess).
> 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)