You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "stack (JIRA)" <ji...@apache.org> on 2013/02/25 21:54:13 UTC

[jira] [Commented] (HBASE-7670) Synchronized operation in CatalogTracker would block handling ZK Event for long time

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

stack commented on HBASE-7670:
------------------------------

Patch won't work because there are a few places where we are synchronizing on metaAvailable; these fellas won't get notified if we figure meta has been assigned.
                
> Synchronized operation in CatalogTracker would block handling ZK Event for long time
> ------------------------------------------------------------------------------------
>
>                 Key: HBASE-7670
>                 URL: https://issues.apache.org/jira/browse/HBASE-7670
>             Project: HBase
>          Issue Type: Bug
>    Affects Versions: 0.94.4
>            Reporter: chunhui shen
>            Assignee: chunhui shen
>            Priority: Critical
>             Fix For: 0.96.0
>
>         Attachments: HBASE-7670.patch, HBASE-7670.patch
>
>
> We found ZK event not be watched by master for a  long time in our testing.
> It seems one ZK-Event-Handle thread block it.
> Attaching some logs on master
> {code}
> 2013-01-16 22:18:55,667 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Handling transition=RS_ZK_REGION_OPENED, 
> 2013-01-16 22:18:56,270 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Handling transition=RS_ZK_REGION_OPENED, 
> ...
> 2013-01-16 23:55:33,259 INFO org.apache.hadoop.hbase.catalog.CatalogTracker: Retrying
> org.apache.hadoop.hbase.client.RetriesExhaustedException: Failed after attempts=100, exceptions:
>         at org.apache.hadoop.hbase.client.ServerCallable.withRetries(ServerCallable.java:183)
>         at org.apache.hadoop.hbase.client.HTable.get(HTable.java:676)
>         at org.apache.hadoop.hbase.catalog.MetaReader.get(MetaReader.java:247)
>         at org.apache.hadoop.hbase.catalog.MetaReader.getRegion(MetaReader.java:349)
>         at org.apache.hadoop.hbase.catalog.MetaReader.readRegionLocation(MetaReader.java:289)
>         at org.apache.hadoop.hbase.catalog.MetaReader.getMetaRegionLocation(MetaReader.java:276)
>         at org.apache.hadoop.hbase.catalog.CatalogTracker.getMetaServerConnection(CatalogTracker.java:424)
>         at org.apache.hadoop.hbase.catalog.CatalogTracker.waitForMeta(CatalogTracker.java:489)
>         at org.apache.hadoop.hbase.catalog.CatalogTracker.waitForMeta(CatalogTracker.java:451)
>         at org.apache.hadoop.hbase.master.handler.ServerShutdownHandler.process(ServerShutdownHandler.java:289)
>         at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:169)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>         at java.lang.Thread.run(Thread.java:662)
> 2013-01-16 23:55:33,261 WARN org.apache.hadoop.hbase.master.AssignmentManager: Attempted to handle region transition for server but server is not online
> {code}
> Between 2013-01-16 22:18:56 and 2013-01-16 23:55:33, there is no any logs about handling ZK Event.
> {code}
> this.metaNodeTracker = new MetaNodeTracker(zookeeper, throwableAborter) {
>       public void nodeDeleted(String path) {
>         if (!path.equals(node)) return;
>         ct.resetMetaLocation();
>       }
>     }
> public void resetMetaLocation() {
>     LOG.debug("Current cached META location, " + metaLocation +
>       ", is not valid, resetting");
>     synchronized(this.metaAvailable) {
>       this.metaAvailable.set(false);
>       this.metaAvailable.notifyAll();
>     }
>   }
> private AdminProtocol getMetaServerConnection(){
> synchronized (metaAvailable){
> ...
> ServerName newLocation = MetaReader.getMetaRegionLocation(this);
> ...
> }
> }
> {code}
> From the above code, we would found that nodeDeleted() would wait synchronized (metaAvailable) until MetaReader.getMetaRegionLocation(this) done,
> however, getMetaRegionLocation() could be retrying for a long time

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira