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

Abhishek Singh Chouhan commented on HBASE-17791:
------------------------------------------------

In HBASE-15251 failover is set to false even when we get an non empty list of 
deadServers
{code}
if (failover) {
      // This may not be a failover actually, especially if meta is on this 
master.
      if (LOG.isDebugEnabled()) {
        LOG.debug("Found dead servers out on cluster " + 
serverManager.getDeadServers());
      }
      // Check if there are any regions on these servers
      failover = false;
....
{code}

Now this is set to true only when one of the many checks like region except 
meta in transition etc. are true which look like valid scenarios , otherwise it 
stays false and we go ahead with assigning regions as per meta. Won't this 
already solve the problem ?



> Locality should not be affected for non-faulty region servers at startup
> ------------------------------------------------------------------------
>
>                 Key: HBASE-17791
>                 URL: https://issues.apache.org/jira/browse/HBASE-17791
>             Project: HBase
>          Issue Type: Improvement
>          Components: Balancer, Region Assignment
>    Affects Versions: 1.3.0, 1.0.3, 1.2.4, 1.1.8
>            Reporter: Lars George
>            Priority: Blocker
>
> We seem to have an issue with store file locality as soon as a single server 
> is missing or faulty upon restart. HBASE-15251 is addressing a subset of the 
> problem, ensuring that some remaining files or an empty server do not trigger 
> a full recovery, but is missing further, similar cases. Especially, the case 
> where a server fails to report in before the active master is finished 
> waiting for them to do so, or where servers have been decomissioned (but not 
> removed from the {{regionservers}} file), and finally the true case of a dead 
> node.
> In case a single node is faulty, the user regions are _not_ assigned as saved 
> in the {{hbase:meta}} table, but completely randomized in a round-robin 
> fashion. An additional factor is that in this case the regions are _not_ 
> assigned to the best matching node (the one with a copy of the data locally), 
> but to any node, leaving the locality in shambles.
> What is also bad, if the {{hbase.hstore.min.locality.to.skip.major.compact}} 
> property is left at the default {{0.0f}}, then an older region that had no 
> writes since the last major compaction happened is just skipped (as expected, 
> usually) and locality stays bad as-is. All reads for those aged-out regions 
> will be network reads. But in any event, having to run a major compaction 
> after a restart is not good anyways.
> The issue is the code in 
> {{AssignmentManager.processDeadServersAndRegionsInTransition()}}, which is 
> handed a list of dead servers. But it immediately sets the {{failover}} flag 
> and the code
> {code}
>     failoverCleanupDone();
>     if (!failover) {
>       // Fresh cluster startup.
>       LOG.info("Clean cluster startup. Don't reassign user regions");
>       assignAllUserRegions(allRegions);
>     } else {
>       LOG.info("Failover! Reassign user regions");
>     }
> {code}
> is not triggering the assignment of the regions to those servers that are 
> still present and have all their region data local. What should happen is 
> that only the missing regions are reassigned, just like in the case of a 
> server failing while the cluster is running.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to