You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Jean-Daniel Cryans (JIRA)" <ji...@apache.org> on 2011/06/21 23:19:47 UTC

[jira] [Created] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Refactor the TimeoutMonitor to make it less racy
------------------------------------------------

                 Key: HBASE-4015
                 URL: https://issues.apache.org/jira/browse/HBASE-4015
             Project: HBase
          Issue Type: Sub-task
    Affects Versions: 0.90.3
            Reporter: Jean-Daniel Cryans
            Priority: Critical
             Fix For: 0.92.0


The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.

The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).

Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13087051#comment-13087051 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

@Stack
Completed the overall changes with OFFLINE state.(I think so) :)
Will upload the patch tomorrow for your review.  Thanks in advance.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13082970#comment-13082970 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

@Stack
Thanks for your review.

{noformat}
The core prob. as per J-D above is that state transitions happen fine out on the regionserver but the master lags processing them; meantime the timeout monitor runs and presumes since its not seen the transition (that is likely in queue to process), it preempts znode setting it OFFLINE.
{noformat}

I would like to clarify one problem here
->Timeout monitor {color:red}DOESNOT{color} preempt an znode to OFFLINE if in PENDING_OPEN state.
' assign(e.getKey(), false, e.getValue());'
Here we pass false for the setOfflineInZK.
If you see the comments in HBASE-3937 JD had pointed out like making this {color:red}'true' will lead to double assignment.{color}

Our soln has been drafted after carefully analysing and reproducing based on JD's comment in HBASE-3937 
and logs from the issue.


I would like to discuss on logs the of HBASE-3937 to be more specific on why the change has been done like this.
{noformat}
RS1 logs
========
2011-05-20 15:48:02,879 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: regionserver:20020-0x3300c164fe0002c Successfully transitioned node d7555a12586e6c788ca55017224b5a51 from M_ZK_REGION_OFFLINE to RS_ZK_REGION_OPENING
2011-05-20 15:48:02,879 DEBUG org.apache.hadoop.hbase.regionserver.HRegion: Opening region: REGION => {NAME => 'ufdr,010066,1305873715825.9361f58931a310a62c15f501ce3261b6.', STARTKEY => '010066', ENDKEY => '010068', ENCODED => 9361f58931a310a62c15f501ce3261b6, TABLE => {{NAME => 'ufdr', FAMILIES => [{NAME => 'value', BLOOMFILTER => 'NONE', REPLICATION_SCOPE => '0', VERSIONS => '3', COMPRESSION => 'GZ', TTL => '432000', BLOCKSIZE => '65536', IN_MEMORY => 'false', BLOCKCACHE => 'true'}]}}
2011-05-20 15:48:02,879 DEBUG org.apache.hadoop.hbase.regionserver.HRegion: Opening region: REGION => {NAME => 'ufdr,001570,1305873689710.d7555a12586e6c788ca55017224b5a51.', STARTKEY => '001570', ENDKEY => '001572', ENCODED => d7555a12586e6c788ca55017224b5a51, TABLE => {{NAME => 'ufdr', FAMILIES => [{NAME => 'value', BLOOMFILTER => 'NONE', REPLICATION_SCOPE => '0', VERSIONS => '3', COMPRESSION => 'GZ', TTL => '432000', BLOCKSIZE => '65536', IN_MEMORY => 'false', BLOCKCACHE => 'true'}]}}
{noformat}
{noformat}


2011-05-20 15:49:58,134 ERROR org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler: Failed open of region=ufdr,010066,1305873715825.9361f58931a310a62c15f501ce3261b6.
java.io.IOException: Exception occured while connecting to the server
	at com.huawei.isap.ump.ha.client.RPCRetryAndSwitchInvoker.retryOperation(RPCRetryAndSwitchInvoker.java:162)
	at com.huawei.isap.ump.ha.client.RPCRetryAndSwitchInvoker.handleFailure(RPCRetryAndSwitchInvoker.java:118)
	at com.huawei.isap.ump.ha.client.RPCRetryAndSwitchInvoker.invoke(RPCRetryAndSwitchInvoker.java:95)
	at $Proxy6.getFileInfo(Unknown Source)
	at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:889)
	at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:724)
	at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:812)
	at org.apache.hadoop.hbase.regionserver.HRegion.checkRegioninfoOnFilesystem(HRegion.java:409)
{noformat}
{noformat}
RS1 logs (Failed to open here)
==============================
2011-05-20 17:00:37,753 WARN org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler: Failed transition from OFFLINE to OPENING for region=9361f58931a310a62c15f501ce3261b6
2011-05-20 17:00:37,753 WARN org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler: Region was hijacked? It no longer exists, encodedName=9361f58931a310a62c15f501ce3261b6
{noformat}
{noformat}
RS2 logs (Failed to open here)
=============================
2011-05-20 16:54:41,385 WARN org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler: Failed transition from OFFLINE to OPENING for region=9361f58931a310a62c15f501ce3261b6
2011-05-20 16:54:41,385 WARN org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler: Region was hijacked? It no longer exists, encodedName=9361f58931a310a62c15f501ce3261b6
{noformat}
{noformat}
RS3 logs (Failed to open here)
==============================
2011-05-20 16:45:29,477 WARN org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler: Failed transition from OFFLINE to OPENING for region=d7555a12586e6c788ca55017224b5a51
2011-05-20 16:45:29,477 WARN org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler: Region was hijacked? It no longer exists, encodedName=d7555a12586e6c788ca55017224b5a51
{noformat}

Here the state of the node is changed from OFFLINE to OPENING.
But before the operation has failed the Timeoutmonitor deducts it but thinks it to be PENDING_OPEN though
the change has happened to RS_OPENING in ZNode.
May be if it had found it to be OPENING then it would have moved it to OFFLINE state in znode.

As master doesnot preempt to OFFLINE and the master in memory state is PENDING_OPEN(Here there is no one to remove the RIT so the timoutmonitor continues to deduct as PENDING_OPEN), everytime this region is considered to be hijacked and no one processes
it.  Even if the request goes to the same RS.

I just want to reiterate the comments given by JD in the defect HBASE-3937.
{noformat}
 It should not create a list of unassigns and assigns, since by the time the list is processed the situation probably changed (I witnessed that a lot).
 This means the action should be taken as we go through first loop.'
{noformat}
 This is what our patch does.  No batch processing is done.  We try to take action as and when we deduct a timeout has occured.
 But again this may not be fool proof. Because there is a chance like as in the above case the timeout
 deducts it to be PENDING_OPEN but as we dont move the node to OFFLINE(In master's memory the state
 is PENDING_OPEN) the RS will say the region is hijacked
 as it cannot transit from OFFLINE to OPENING and hence the problem prevails.


 Coming back to JD's comments again
{noformat}
 One of the major issues is the lack of atomicity, so any action taken should first check the current state, keep the version number, decide of the corrective measure and update the znode by expecting the version it first got.
 {noformat}

 Now comes the issue of how to know the exact state that the znode is currently in and what action to take.
 If we again move to OFFLINE then there may be problem of double assignment.
 So we need to manage with some versions as JD told.  
 Here instead of versions we opted for a new state following are the reasons
 {noformat}
	// Initialize the znode version.
      this.version =
        ZKAssign.transitionNodeOpening(server.getZooKeeper(),
          regionInfo, server.getServerName());
 {noformat}
 ->RS doesnt have any previous version history here.  So comparing the new OFFLINE state and the
 prev offline state may be tricky.(needs some tweaking)
 ->Introducing an intermediate state here would bring more clarity to the code and system.
 Thats why we planned to introduce RE_ALLOCATE. Adding the servername is an additional precaution.
{noformat}
If the updating of the znode is successful, we know for sure that the operation will be seen by the region servers.
{noformat}


 So now what we do is deduct timeout, try moving the state of znode to a RE_ALLOCATE.
 If really successfull all the RS will know that some updation has happened.
 Now another RS(or same RS) has the chance to operate on this new state and will not say as hijacked.
{noformat}
If it's not successful, the situation needs to be reassessed.
{noformat}
 If changing state to RE_ALLOCATE is not successful then what. Now master is aware the RS has operated on the region and changed it to another state may to OPENING or OPENED.

 As we cannot move the state to OFFLINE in zknode we are forced to have some mechanism between
 the RS and master to handle this problem.  Hence the new state RE_ALLOCATE came into
 picture.

 Thus our current implementation not only address the time lag but also clear atomicity is maintained.

Stack, am i clear in my explanation?
Actually before proposing the soln we went thro JD's comments analysed the logs and then we wnated
to take care of all the comments as per JD and infact reproduced all the problems.


> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13100573#comment-13100573 ] 

stack commented on HBASE-4015:
------------------------------

Sorry.  I bungled the commit.  Ram pointed out that the latest attached here is not the right patch to apply.  I should have gotten the patch from RB.  I had to make two attempts at fixup.  My third application hopefully is correct (I'm sure Ram will let me know if it is not).  Thanks for staying on top of this Ram.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13072162#comment-13072162 ] 

Ted Yu commented on HBASE-4015:
-------------------------------

Shall we punt this to 0.94 ?

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Priority: Critical
>             Fix For: 0.92.0
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13090031#comment-13090031 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

Thanks for your comments Ted and Stack.
{bq}Why the special handling of meta regions here:
This is for the issue HBASE-4203.  I should have commented in this JIRA also about this. 
All the formatting issues i will address it.
I will explain in detail the major race conditions and which parts of PATCH-2 handles corresponding scenarios.






> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Jonathan Gray (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13084333#comment-13084333 ] 

Jonathan Gray commented on HBASE-4015:
--------------------------------------

Sorry I'm a little late to this discussion but I like the idea of not adding a new state.  Instead, we can just pass the znode version number in the RPC to the regionservers.  Or encode the servername in the znode.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13091795#comment-13091795 ] 

jiraposter@reviews.apache.org commented on HBASE-4015:
------------------------------------------------------


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/1668/
-----------------------------------------------------------

(Updated 2011-08-26 14:23:24.757551)


Review request for Ted Yu, Michael Stack, Jean-Daniel Cryans, and Jonathan Gray.


Summary (updated)
-------

HBASE-4015 - updated patch.


        invokeTimeOutManager(regionState.getRegion(),
            TimeOutOperationType.ASSIGN);

Instead of storing the state when timeout was deducted we process it
using future task.
================================================================

      case RS_ZK_REGION_OPENING:
        // TODO: Could check if it was on deadServers.  If it was, then we could
        // do what happens in TimeoutMonitor when it sees this condition.

        // Just insert region into RIT
        // If this never updates the timeout will trigger new assignment
        if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
          regionsInTransition.put(encodedRegionName, new RegionState(
              regionInfo, RegionState.State.OPENING, data.getStamp(), data
                  .getOrigin()));
          processOpeningState(regionInfo);
          break;
        }
        regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
            RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
        break;

This change is for HBASE-4203. META and ROOT table need not wait till timeout
======================================================================

In forceRegionStateToOffline()

    } else {
      // If invoked from timeout monitor donot force it to OFFLINE. Based on the
      // state we will decide if to change in-memory state to OFFLINE or not.  It will
      // be done before setting the znode to OFFLINE state.
      if (!timeOutMonitorReAllocate) {
        LOG.debug("Forcing OFFLINE; was=" + state);
        state.update(RegionState.State.OFFLINE);
      }
If the timeout monitor tries to reallcoate the node then dont make
the inmemory state to OFFLINE.
But the noraml assign flow doesnot expect the inmemory state to OFFLINE.
Hence the above change.  This is continued with the check in 
assign()
int setOfflineInZooKeeper(final RegionState state,
      boolean timeOutMonitorReAllocate) {
    // If invoked from timeoutmonitor the current state in memory need not be
    // OFFLINE.  
    if (!timeOutMonitorReAllocate && !state.isClosed() && !state.isOffline()) {
          this.master.abort("Unexpected state trying to OFFLINE; " + state,
          new IllegalStateException());
      return -1;
    }
======================================================================

    boolean allowCreation = false;
    // If the isReAllocate is true and the current state is PENDING_OPEN
    // or OPENING then update the inmemory state to PENDING_OPEN. This is
    // important because
    // if timeoutmonitor deducts that a region was in OPENING state for a long
    // time but by the
    // time timeout monitor tranits the node to OFFLINE the RS would have opened
    // the node and the
    // state in znode will be RS_ZK_REGION_OPENED. Inorder to invoke the
    // OpenedRegionHandler
    // we expect the inmemeory state to be PENDING_OPEN or OPENING.
    // For all other cases we can change the inmemory state to OFFLINE.
    if (timeOutMonitorReAllocate
        && (state.getState().equals(RegionState.State.PENDING_OPEN) || state
            .getState().equals(RegionState.State.OPENING))) {
      state.update(RegionState.State.PENDING_OPEN);
      allowCreation = false;
    } else {
      state.update(RegionState.State.OFFLINE);
      allowCreation = true;
    }
This change is quite tricky.  
In normal assign flow the unassigned node for the region will not be present
Hence we need to allow the creation of the node newly.
But in timeout monitor case we will have the node present in some state hence 
we decide whether to create node newly or not inside ZKAssign.createOrForceNodeOffline

The above code also updates the inmemory state of OFFLINE or PENDING_OPEN
which was not update in the previous forceRegionStateToOffline() call.
==============================================================================
In ZKAssign.java()
    if (version == -1) {
      // If timeoutmonitor deducts a node to be in OPENING state but before it
      // could
      // transit to OFFLINE state if RS had opened the region then the Master
      // deletes the
      // assigned region znode. In that case the znode will not exist. So we
      // should not
      // create the znode again which will lead to double assignment.
      if (timeOutMonitorReAllocate && !allowCreation) {
        return -1;
      }
this part prevents double assignment
If timeoutmonitor tries to force to OFFLINE state an existing region which was in RIT
but before it could do that if the node was opened then openedregionhandler will delete
the node hence we should not create the node again.

=======================================================================================
In ZkAssign.java()
    } else {
      RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
          .getEncodedName(), stat);
      // Do not move the node to OFFLINE if znode is in any of the following
      // state.
      // Because these are already executed states.
      if (timeOutMonitorReAllocate && null != curDataInZNode) {
        EventType eventType = curDataInZNode.getEventType();
        if (eventType.equals(EventType.RS_ZK_REGION_CLOSING)
            || eventType.equals(EventType.RS_ZK_REGION_CLOSED)
            || eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
          return -1;
        }
      }
This check prevents from moving the node to OFFLINE state if just before
the node is tried to force to OFFLINE the RS would have changed the state 
to either CLOSING or CLOSED or OPENED.  now again moving to OFFLINE
will lead to douoble assignment and will an additional operaition.

====================================================================================
In ZKassign.java()
      boolean setData = false;
      try {
        setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
        // Setdata throws KeeperException which aborts the Master. So we are
        // catching it here.
        // If just before setting the znode to OFFLINE if the RS has made any
        // change to the
        // znode state then we need to return -1.
      } catch (KeeperException kpe) {
        LOG.info("Version mismatch while setting the node to OFFLINE state.");
        return -1;
      }
This change is actually to avoid the master from abortng. If the forceful OFFLINE is in
progrss but just before setting the RS has changed the state.
Then the setData will fail leading to master abort.
Hence we are catching the exception.
====================================================================================

In assignmentManager.java
+      if (setOfflineInZK && versionOfOfflineNode == -1)
+        return;
This is nothing but the refactoring done in the existing code.
-     if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
So if setting the version is unsuccessful return.
=====================================================================================
In ZKassign.java()
// the below check ensure that double assignment doesnot happen.
    // When the node is created for the first time then the expected version
    // that is
    // passed will be -1 and the version in znode will be 0.
    // In all other cases the version in znode will be > 0.
    else if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
        && endState.equals(EventType.RS_ZK_REGION_OPENING)
        && expectedVersion == -1 && stat.getVersion() != 0) {
      LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
          + encoded + " from " + beginState + " to " + endState + " failed, "
          + "the node existed but was version " + stat.getVersion()
          + " not the expected version " + expectedVersion));
      return -1;
    }
As the comment explains when the node is created for first time the expectedversion will
be -1 but the actual version will be 0.  Here the scenario is
If RS1 has not tranitioned the node from OFFLINE to OPENING if
RS2 gets the call from Master after forcefully chaning to OFFLINE the
Rs2 will take the control of the node.
At that time if RS1 starts transmitting the node then we should not allow
it.
===============================================================================


This addresses bug HBASE-4015.
    https://issues.apache.org/jira/browse/HBASE-4015


Diffs
-----

  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java 1161985 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java 1161985 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 1161985 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java 1161985 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 1161985 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1161985 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java PRE-CREATION 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1161985 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java 1161985 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java 1161985 

Diff: https://reviews.apache.org/r/1668/diff


Testing
-------

Yes.  But could not add new test case.
TestMasterFailOver is passing with the current changes also.


Thanks,

ramkrishna



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13100737#comment-13100737 ] 

Hudson commented on HBASE-4015:
-------------------------------

Integrated in HBase-TRUNK #2190 (See [https://builds.apache.org/job/HBase-TRUNK/2190/])
    HBASE-4105 HBASE-4015-Making the timeout monitor less racy; third attempt
HBASE-4015 Refactor the TimeoutMonitor to make it less racy -- REVERT.  Ram says I have applied wrong patch.  Redoing.   A SECOND TIMEsvn rm --force   src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.javasvn rm --force   src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java
HBASE-4015 Refactor the TimeoutMonitor to make it less racy -- reapply with HBASE-4015_reprepared_trunk_2.patch
HBASE-4015 Refactor the TimeoutMonitor to make it less racy -- REVERT.  Ram says I have applied wrong patch.  Redoing.

stack : 
Files : 
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignCallable.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/UnAssignCallable.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java

stack : 
Files : 
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java

stack : 
Files : 
* /hbase/trunk/CHANGES.txt
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java

stack : 
Files : 
* /hbase/trunk/CHANGES.txt
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java


> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13082353#comment-13082353 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

I have not written any testcases to cover this.  Planning to write a testcase.  hopefully will write one covering the scenarios.


> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13090026#comment-13090026 ] 

stack commented on HBASE-4015:
------------------------------

bq. In TRUNK, HRegionInfo.isMetaRegion() only checks for .META.

Thanks.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13092317#comment-13092317 ] 

jiraposter@reviews.apache.org commented on HBASE-4015:
------------------------------------------------------


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/1668/#review1672
-----------------------------------------------------------



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
<https://reviews.apache.org/r/1668/#comment3807>

    The three states should be placed on three lines with proper indentation.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3808>

    versionOfTheOffline seems to be copied from a variable name. Please split it into 4 words: version of the offline


- Ted


On 2011-08-27 11:51:16, ramkrishna vasudevan wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/1668/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2011-08-27 11:51:16)
bq.  
bq.  
bq.  Review request for Ted Yu, Michael Stack, Jean-Daniel Cryans, and Jonathan Gray.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  HBASE-4015 - updated patch.
bq.  
bq.  
bq.          invokeTimeOutManager(regionState.getRegion(),
bq.              TimeOutOperationType.ASSIGN);
bq.  
bq.  Instead of storing the state when timeout was deducted we process it
bq.  using future task.
bq.  ================================================================
bq.  
bq.        case RS_ZK_REGION_OPENING:
bq.          // TODO: Could check if it was on deadServers.  If it was, then we could
bq.          // do what happens in TimeoutMonitor when it sees this condition.
bq.  
bq.          // Just insert region into RIT
bq.          // If this never updates the timeout will trigger new assignment
bq.          if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
bq.            regionsInTransition.put(encodedRegionName, new RegionState(
bq.                regionInfo, RegionState.State.OPENING, data.getStamp(), data
bq.                    .getOrigin()));
bq.            processOpeningState(regionInfo);
bq.            break;
bq.          }
bq.          regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
bq.              RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
bq.          break;
bq.  
bq.  This change is for HBASE-4203. META and ROOT table need not wait till timeout
bq.  ======================================================================
bq.  
bq.  In forceRegionStateToOffline()
bq.  
bq.      } else {
bq.        // If invoked from timeout monitor donot force it to OFFLINE. Based on the
bq.        // state we will decide if to change in-memory state to OFFLINE or not.  It will
bq.        // be done before setting the znode to OFFLINE state.
bq.        if (!timeOutMonitorReAllocate) {
bq.          LOG.debug("Forcing OFFLINE; was=" + state);
bq.          state.update(RegionState.State.OFFLINE);
bq.        }
bq.  If the timeout monitor tries to reallcoate the node then dont make
bq.  the inmemory state to OFFLINE.
bq.  But the noraml assign flow doesnot expect the inmemory state to OFFLINE.
bq.  Hence the above change.  This is continued with the check in 
bq.  assign()
bq.  int setOfflineInZooKeeper(final RegionState state,
bq.        boolean timeOutMonitorReAllocate) {
bq.      // If invoked from timeoutmonitor the current state in memory need not be
bq.      // OFFLINE.  
bq.      if (!timeOutMonitorReAllocate && !state.isClosed() && !state.isOffline()) {
bq.            this.master.abort("Unexpected state trying to OFFLINE; " + state,
bq.            new IllegalStateException());
bq.        return -1;
bq.      }
bq.  ======================================================================
bq.  
bq.      boolean allowCreation = false;
bq.      // If the isReAllocate is true and the current state is PENDING_OPEN
bq.      // or OPENING then update the inmemory state to PENDING_OPEN. This is
bq.      // important because
bq.      // if timeoutmonitor deducts that a region was in OPENING state for a long
bq.      // time but by the
bq.      // time timeout monitor tranits the node to OFFLINE the RS would have opened
bq.      // the node and the
bq.      // state in znode will be RS_ZK_REGION_OPENED. Inorder to invoke the
bq.      // OpenedRegionHandler
bq.      // we expect the inmemeory state to be PENDING_OPEN or OPENING.
bq.      // For all other cases we can change the inmemory state to OFFLINE.
bq.      if (timeOutMonitorReAllocate
bq.          && (state.getState().equals(RegionState.State.PENDING_OPEN) || state
bq.              .getState().equals(RegionState.State.OPENING))) {
bq.        state.update(RegionState.State.PENDING_OPEN);
bq.        allowCreation = false;
bq.      } else {
bq.        state.update(RegionState.State.OFFLINE);
bq.        allowCreation = true;
bq.      }
bq.  This change is quite tricky.  
bq.  In normal assign flow the unassigned node for the region will not be present
bq.  Hence we need to allow the creation of the node newly.
bq.  But in timeout monitor case we will have the node present in some state hence 
bq.  we decide whether to create node newly or not inside ZKAssign.createOrForceNodeOffline
bq.  
bq.  The above code also updates the inmemory state of OFFLINE or PENDING_OPEN
bq.  which was not update in the previous forceRegionStateToOffline() call.
bq.  ==============================================================================
bq.  In ZKAssign.java()
bq.      if (version == -1) {
bq.        // If timeoutmonitor deducts a node to be in OPENING state but before it
bq.        // could
bq.        // transit to OFFLINE state if RS had opened the region then the Master
bq.        // deletes the
bq.        // assigned region znode. In that case the znode will not exist. So we
bq.        // should not
bq.        // create the znode again which will lead to double assignment.
bq.        if (timeOutMonitorReAllocate && !allowCreation) {
bq.          return -1;
bq.        }
bq.  this part prevents double assignment
bq.  If timeoutmonitor tries to force to OFFLINE state an existing region which was in RIT
bq.  but before it could do that if the node was opened then openedregionhandler will delete
bq.  the node hence we should not create the node again.
bq.  
bq.  =======================================================================================
bq.  In ZkAssign.java()
bq.      } else {
bq.        RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
bq.            .getEncodedName(), stat);
bq.        // Do not move the node to OFFLINE if znode is in any of the following
bq.        // state.
bq.        // Because these are already executed states.
bq.        if (timeOutMonitorReAllocate && null != curDataInZNode) {
bq.          EventType eventType = curDataInZNode.getEventType();
bq.          if (eventType.equals(EventType.RS_ZK_REGION_CLOSING)
bq.              || eventType.equals(EventType.RS_ZK_REGION_CLOSED)
bq.              || eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
bq.            return -1;
bq.          }
bq.        }
bq.  This check prevents from moving the node to OFFLINE state if just before
bq.  the node is tried to force to OFFLINE the RS would have changed the state 
bq.  to either CLOSING or CLOSED or OPENED.  now again moving to OFFLINE
bq.  will lead to douoble assignment and will an additional operaition.
bq.  
bq.  ====================================================================================
bq.  In ZKassign.java()
bq.        boolean setData = false;
bq.        try {
bq.          setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
bq.          // Setdata throws KeeperException which aborts the Master. So we are
bq.          // catching it here.
bq.          // If just before setting the znode to OFFLINE if the RS has made any
bq.          // change to the
bq.          // znode state then we need to return -1.
bq.        } catch (KeeperException kpe) {
bq.          LOG.info("Version mismatch while setting the node to OFFLINE state.");
bq.          return -1;
bq.        }
bq.  This change is actually to avoid the master from abortng. If the forceful OFFLINE is in
bq.  progrss but just before setting the RS has changed the state.
bq.  Then the setData will fail leading to master abort.
bq.  Hence we are catching the exception.
bq.  ====================================================================================
bq.  
bq.  In assignmentManager.java
bq.  +      if (setOfflineInZK && versionOfOfflineNode == -1)
bq.  +        return;
bq.  This is nothing but the refactoring done in the existing code.
bq.  -     if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
bq.  So if setting the version is unsuccessful return.
bq.  =====================================================================================
bq.  In ZKassign.java()
bq.  // the below check ensure that double assignment doesnot happen.
bq.      // When the node is created for the first time then the expected version
bq.      // that is
bq.      // passed will be -1 and the version in znode will be 0.
bq.      // In all other cases the version in znode will be > 0.
bq.      else if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
bq.          && endState.equals(EventType.RS_ZK_REGION_OPENING)
bq.          && expectedVersion == -1 && stat.getVersion() != 0) {
bq.        LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
bq.            + encoded + " from " + beginState + " to " + endState + " failed, "
bq.            + "the node existed but was version " + stat.getVersion()
bq.            + " not the expected version " + expectedVersion));
bq.        return -1;
bq.      }
bq.  As the comment explains when the node is created for first time the expectedversion will
bq.  be -1 but the actual version will be 0.  Here the scenario is
bq.  If RS1 has not tranitioned the node from OFFLINE to OPENING if
bq.  RS2 gets the call from Master after forcefully chaning to OFFLINE the
bq.  Rs2 will take the control of the node.
bq.  At that time if RS1 starts transmitting the node then we should not allow
bq.  it.
bq.  ===============================================================================
bq.  
bq.  
bq.  This addresses bug HBASE-4015.
bq.      https://issues.apache.org/jira/browse/HBASE-4015
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java PRE-CREATION 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java 1162313 
bq.  
bq.  Diff: https://reviews.apache.org/r/1668/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  Yes.  But could not add new test case.
bq.  TestMasterFailOver is passing with the current changes also.
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  ramkrishna
bq.  
bq.



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13080402#comment-13080402 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

@Ted,
I got your point.  Actually we are planning to lookup once again after setting any state so that whatever was the intended state that has really been set in the ZK.
If master tries to change to RE_ALLOCATE it will issue the command to ZK.  By the time the RS would have changed it to OPENING.

Now the master will once again check if the state is RE_ALLOCATE .  If yes the operation is successful if not(RS has changed to OPENING) the master will update his inmemory state to OPENING and will wait on this state to change.
Similar is the case with RS.

(Even checking for the version also can be done by comparing with the version that was got from ZK and the version that the Master or RS has really intended to set).

In one of the ZK LeaderElection algo we did something similar to this.  One guy will create a sequential node.  Will know what is the node he created.
If someother guy at the same time has created another sequential node in the same path just before the first guy had created, the first guy's node id will be greater than the second guy apparently the second guy wins the race.

Is it fine Ted? Do correct me if this is wrong.  Also if i could figure something better i would post it.  Thanks Ted.


> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13090032#comment-13090032 ] 

Ted Yu commented on HBASE-4015:
-------------------------------

At line 1098 in AssignmentManager.java, calling assign(state, setOfflineInZK, forceNewPlan, isReAllocate) would suffice:
{code}
      if (!isReAllocate) {
        assign(state, setOfflineInZK, forceNewPlan);
      } else {
        assign(state, setOfflineInZK, forceNewPlan, isReAllocate);
      }
{code}



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13085543#comment-13085543 ] 

stack commented on HBASE-4015:
------------------------------

bq. So whats your idea Stack? Can i start digging as how many changes do we need to make if we go with OFFLINE state and what are the interface changes etc.

That sounds good to me.  I took a look and what I saw was that setting OFFLINE state, its currently not easy getting back the znode seqid; might have to add something here.  Then, the seqid would have to be passed over the rpc when we do open region.  I'd say add a new open region method, one that takes two args -- the region name and the seqid.. leave the old one in place and use -1 or something to flag an open where no seqid has been passed (maybe shell wants to do an open region and it won't have the seqid).  Then I'd pass the seqid down into the openhandler....  Then use it checking the znode seqid when we check OFFLINE.

Something like that.

Good on you Ram.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13090034#comment-13090034 ] 

Ted Yu commented on HBASE-4015:
-------------------------------

@Ramkrishna:
Can you publish patch 2 on reviewboard for further comments ?

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13087120#comment-13087120 ] 

stack commented on HBASE-4015:
------------------------------

@Ram You are a good man.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13081859#comment-13081859 ] 

stack commented on HBASE-4015:
------------------------------

I like this diagram of yours Ram.

Do we need new RS_ALLOCATE state?  Could we just have OFFLINE plus your suggestion of adding RS name so its OFFLINE+RS_TO_OPEN_REGION_ON?  What happens if we assign the region back to RS1 (it can happen).

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13084060#comment-13084060 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

@Stack,
Was seeing the possibility of using OFFLINE state.  Thought of few things
-> Now we need to change behaviour in all the cases in timeoutmonitor to preempt the node to OFFLINE with RS name.
->Before changing to OFFLINE see what is the state in RS.  If still OFFLINE/OPENING change it to OFFLINE+Servername address
->After changing it to OFFLINE get the latest version and pass it to the RS from Master which inturn goes to the OpenRegionHandler.  
->This will be needed when we transit from OFFLINE to OPENING to ensure whether the current transition from OFFLINE to OPENING is for timeout call or previous OFFLINE to OPENING did not happen.
->also the servername is necessary to avoid processing of the transition by the RS who is no longer owner of the znode.
->And even in normal flow(normal assign flow) we need to add the servername of RS along with OFFLINE who will process the unassigned node

These will be the highlevel changes that we need to make in the current patch if we need to avoid the new state.  


> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13098995#comment-13098995 ] 

jiraposter@reviews.apache.org commented on HBASE-4015:
------------------------------------------------------



bq.  On 2011-09-07 10:13:44, ramkrishna vasudevan wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 415
bq.  > <https://reviews.apache.org/r/1668/diff/3/?file=36181#file36181line415>
bq.  >
bq.  >     If you see the implementation of processOpeningState it is same as the handling of the previous OPENING state handling by timeout monitor.   Just that now we call it immediately instead of timeout monitor acting on it.
bq.  >     
bq.  >     "Should we at least check the znode again before going ahead with assign? "
bq.  >     
bq.  >     Now for the possibility of the znode moving to OPENED state before we try to assign to a new RS, our new logic of mulitiple checks that we do before a node can be forcefully moved to OFFLINE will handle the scenario that you have told.

ok


bq.  On 2011-09-07 10:13:44, ramkrishna vasudevan wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1260
bq.  > <https://reviews.apache.org/r/1668/diff/3/?file=36181#file36181line1260>
bq.  >
bq.  >     formatting done and the variable name has been changed from reassign to hijackAndPreempt.
bq.  >     
bq.  >     Stack last time you asked to change the variable name to reassign :)).

Sorry if I did.  I thought I was commenting on use of word 'realloc' when it seemed like you meant reassign.  I think I spent more time reviewing this time than I did last time.


bq.  On 2011-09-07 10:13:44, ramkrishna vasudevan wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 2335
bq.  > <https://reviews.apache.org/r/1668/diff/3/?file=36181#file36181line2335>
bq.  >
bq.  >     Next time will not repeat.  When we refactor a piece of code to a new api and apply format on that the formatter does this.
bq.  >     Sometimes it gets missed out stack. Sorry

No worries.  For next time.


- Michael


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/1668/#review1783
-----------------------------------------------------------


On 2011-08-29 12:11:23, ramkrishna vasudevan wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/1668/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2011-08-29 12:11:23)
bq.  
bq.  
bq.  Review request for Ted Yu, Michael Stack, Jean-Daniel Cryans, and Jonathan Gray.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  HBASE-4015 - updated patch.
bq.  
bq.  
bq.          invokeTimeOutManager(regionState.getRegion(),
bq.              TimeOutOperationType.ASSIGN);
bq.  
bq.  Instead of storing the state when timeout was deducted we process it
bq.  using future task.
bq.  ================================================================
bq.  
bq.        case RS_ZK_REGION_OPENING:
bq.          // TODO: Could check if it was on deadServers.  If it was, then we could
bq.          // do what happens in TimeoutMonitor when it sees this condition.
bq.  
bq.          // Just insert region into RIT
bq.          // If this never updates the timeout will trigger new assignment
bq.          if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
bq.            regionsInTransition.put(encodedRegionName, new RegionState(
bq.                regionInfo, RegionState.State.OPENING, data.getStamp(), data
bq.                    .getOrigin()));
bq.            processOpeningState(regionInfo);
bq.            break;
bq.          }
bq.          regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
bq.              RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
bq.          break;
bq.  
bq.  This change is for HBASE-4203. META and ROOT table need not wait till timeout
bq.  ======================================================================
bq.  
bq.  In forceRegionStateToOffline()
bq.  
bq.      } else {
bq.        // If invoked from timeout monitor donot force it to OFFLINE. Based on the
bq.        // state we will decide if to change in-memory state to OFFLINE or not.  It will
bq.        // be done before setting the znode to OFFLINE state.
bq.        if (!timeOutMonitorReAllocate) {
bq.          LOG.debug("Forcing OFFLINE; was=" + state);
bq.          state.update(RegionState.State.OFFLINE);
bq.        }
bq.  If the timeout monitor tries to reallcoate the node then dont make
bq.  the inmemory state to OFFLINE.
bq.  But the noraml assign flow doesnot expect the inmemory state to OFFLINE.
bq.  Hence the above change.  This is continued with the check in 
bq.  assign()
bq.  int setOfflineInZooKeeper(final RegionState state,
bq.        boolean timeOutMonitorReAllocate) {
bq.      // If invoked from timeoutmonitor the current state in memory need not be
bq.      // OFFLINE.  
bq.      if (!timeOutMonitorReAllocate && !state.isClosed() && !state.isOffline()) {
bq.            this.master.abort("Unexpected state trying to OFFLINE; " + state,
bq.            new IllegalStateException());
bq.        return -1;
bq.      }
bq.  ======================================================================
bq.  
bq.      boolean allowCreation = false;
bq.      // If the isReAllocate is true and the current state is PENDING_OPEN
bq.      // or OPENING then update the inmemory state to PENDING_OPEN. This is
bq.      // important because
bq.      // if timeoutmonitor deducts that a region was in OPENING state for a long
bq.      // time but by the
bq.      // time timeout monitor tranits the node to OFFLINE the RS would have opened
bq.      // the node and the
bq.      // state in znode will be RS_ZK_REGION_OPENED. Inorder to invoke the
bq.      // OpenedRegionHandler
bq.      // we expect the inmemeory state to be PENDING_OPEN or OPENING.
bq.      // For all other cases we can change the inmemory state to OFFLINE.
bq.      if (timeOutMonitorReAllocate
bq.          && (state.getState().equals(RegionState.State.PENDING_OPEN) || state
bq.              .getState().equals(RegionState.State.OPENING))) {
bq.        state.update(RegionState.State.PENDING_OPEN);
bq.        allowCreation = false;
bq.      } else {
bq.        state.update(RegionState.State.OFFLINE);
bq.        allowCreation = true;
bq.      }
bq.  This change is quite tricky.  
bq.  In normal assign flow the unassigned node for the region will not be present
bq.  Hence we need to allow the creation of the node newly.
bq.  But in timeout monitor case we will have the node present in some state hence 
bq.  we decide whether to create node newly or not inside ZKAssign.createOrForceNodeOffline
bq.  
bq.  The above code also updates the inmemory state of OFFLINE or PENDING_OPEN
bq.  which was not update in the previous forceRegionStateToOffline() call.
bq.  ==============================================================================
bq.  In ZKAssign.java()
bq.      if (version == -1) {
bq.        // If timeoutmonitor deducts a node to be in OPENING state but before it
bq.        // could
bq.        // transit to OFFLINE state if RS had opened the region then the Master
bq.        // deletes the
bq.        // assigned region znode. In that case the znode will not exist. So we
bq.        // should not
bq.        // create the znode again which will lead to double assignment.
bq.        if (timeOutMonitorReAllocate && !allowCreation) {
bq.          return -1;
bq.        }
bq.  this part prevents double assignment
bq.  If timeoutmonitor tries to force to OFFLINE state an existing region which was in RIT
bq.  but before it could do that if the node was opened then openedregionhandler will delete
bq.  the node hence we should not create the node again.
bq.  
bq.  =======================================================================================
bq.  In ZkAssign.java()
bq.      } else {
bq.        RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
bq.            .getEncodedName(), stat);
bq.        // Do not move the node to OFFLINE if znode is in any of the following
bq.        // state.
bq.        // Because these are already executed states.
bq.        if (timeOutMonitorReAllocate && null != curDataInZNode) {
bq.          EventType eventType = curDataInZNode.getEventType();
bq.          if (eventType.equals(EventType.RS_ZK_REGION_CLOSING)
bq.              || eventType.equals(EventType.RS_ZK_REGION_CLOSED)
bq.              || eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
bq.            return -1;
bq.          }
bq.        }
bq.  This check prevents from moving the node to OFFLINE state if just before
bq.  the node is tried to force to OFFLINE the RS would have changed the state 
bq.  to either CLOSING or CLOSED or OPENED.  now again moving to OFFLINE
bq.  will lead to douoble assignment and will an additional operaition.
bq.  
bq.  ====================================================================================
bq.  In ZKassign.java()
bq.        boolean setData = false;
bq.        try {
bq.          setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
bq.          // Setdata throws KeeperException which aborts the Master. So we are
bq.          // catching it here.
bq.          // If just before setting the znode to OFFLINE if the RS has made any
bq.          // change to the
bq.          // znode state then we need to return -1.
bq.        } catch (KeeperException kpe) {
bq.          LOG.info("Version mismatch while setting the node to OFFLINE state.");
bq.          return -1;
bq.        }
bq.  This change is actually to avoid the master from abortng. If the forceful OFFLINE is in
bq.  progrss but just before setting the RS has changed the state.
bq.  Then the setData will fail leading to master abort.
bq.  Hence we are catching the exception.
bq.  ====================================================================================
bq.  
bq.  In assignmentManager.java
bq.  +      if (setOfflineInZK && versionOfOfflineNode == -1)
bq.  +        return;
bq.  This is nothing but the refactoring done in the existing code.
bq.  -     if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
bq.  So if setting the version is unsuccessful return.
bq.  =====================================================================================
bq.  In ZKassign.java()
bq.  // the below check ensure that double assignment doesnot happen.
bq.      // When the node is created for the first time then the expected version
bq.      // that is
bq.      // passed will be -1 and the version in znode will be 0.
bq.      // In all other cases the version in znode will be > 0.
bq.      else if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
bq.          && endState.equals(EventType.RS_ZK_REGION_OPENING)
bq.          && expectedVersion == -1 && stat.getVersion() != 0) {
bq.        LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
bq.            + encoded + " from " + beginState + " to " + endState + " failed, "
bq.            + "the node existed but was version " + stat.getVersion()
bq.            + " not the expected version " + expectedVersion));
bq.        return -1;
bq.      }
bq.  As the comment explains when the node is created for first time the expectedversion will
bq.  be -1 but the actual version will be 0.  Here the scenario is
bq.  If RS1 has not tranitioned the node from OFFLINE to OPENING if
bq.  RS2 gets the call from Master after forcefully chaning to OFFLINE the
bq.  Rs2 will take the control of the node.
bq.  At that time if RS1 starts transmitting the node then we should not allow
bq.  it.
bq.  ===============================================================================
bq.  
bq.  
bq.  This addresses bug HBASE-4015.
bq.      https://issues.apache.org/jira/browse/HBASE-4015
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java PRE-CREATION 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java 1161985 
bq.  
bq.  Diff: https://reviews.apache.org/r/1668/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  Yes.  But could not add new test case.
bq.  TestMasterFailOver is passing with the current changes also.
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  ramkrishna
bq.  
bq.



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13082463#comment-13082463 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

Thanks for the comments Ted.
sorry for missing this line wrap :(
One executor service is part of hbase, the other one is needed for future task execution.  I went through the code of ExecutorService in hbase.  It was used to do some eventhandling.  
So i thought of using Java threadpool executor service.


> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13082974#comment-13082974 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

Stack

After seeing the explanation if you still feel OFFLINE state+RS address will be better, i will dig in more into it and check the value addition and let you know.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13085521#comment-13085521 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

@Jonathan
Thanks for your comments.
I will implement the changes so that we dont have a new state and will upload a new patch asap.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13092281#comment-13092281 ] 

jiraposter@reviews.apache.org commented on HBASE-4015:
------------------------------------------------------


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/1668/
-----------------------------------------------------------

(Updated 2011-08-27 11:51:16.491607)


Review request for Ted Yu, Michael Stack, Jean-Daniel Cryans, and Jonathan Gray.


Summary
-------

HBASE-4015 - updated patch.


        invokeTimeOutManager(regionState.getRegion(),
            TimeOutOperationType.ASSIGN);

Instead of storing the state when timeout was deducted we process it
using future task.
================================================================

      case RS_ZK_REGION_OPENING:
        // TODO: Could check if it was on deadServers.  If it was, then we could
        // do what happens in TimeoutMonitor when it sees this condition.

        // Just insert region into RIT
        // If this never updates the timeout will trigger new assignment
        if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
          regionsInTransition.put(encodedRegionName, new RegionState(
              regionInfo, RegionState.State.OPENING, data.getStamp(), data
                  .getOrigin()));
          processOpeningState(regionInfo);
          break;
        }
        regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
            RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
        break;

This change is for HBASE-4203. META and ROOT table need not wait till timeout
======================================================================

In forceRegionStateToOffline()

    } else {
      // If invoked from timeout monitor donot force it to OFFLINE. Based on the
      // state we will decide if to change in-memory state to OFFLINE or not.  It will
      // be done before setting the znode to OFFLINE state.
      if (!timeOutMonitorReAllocate) {
        LOG.debug("Forcing OFFLINE; was=" + state);
        state.update(RegionState.State.OFFLINE);
      }
If the timeout monitor tries to reallcoate the node then dont make
the inmemory state to OFFLINE.
But the noraml assign flow doesnot expect the inmemory state to OFFLINE.
Hence the above change.  This is continued with the check in 
assign()
int setOfflineInZooKeeper(final RegionState state,
      boolean timeOutMonitorReAllocate) {
    // If invoked from timeoutmonitor the current state in memory need not be
    // OFFLINE.  
    if (!timeOutMonitorReAllocate && !state.isClosed() && !state.isOffline()) {
          this.master.abort("Unexpected state trying to OFFLINE; " + state,
          new IllegalStateException());
      return -1;
    }
======================================================================

    boolean allowCreation = false;
    // If the isReAllocate is true and the current state is PENDING_OPEN
    // or OPENING then update the inmemory state to PENDING_OPEN. This is
    // important because
    // if timeoutmonitor deducts that a region was in OPENING state for a long
    // time but by the
    // time timeout monitor tranits the node to OFFLINE the RS would have opened
    // the node and the
    // state in znode will be RS_ZK_REGION_OPENED. Inorder to invoke the
    // OpenedRegionHandler
    // we expect the inmemeory state to be PENDING_OPEN or OPENING.
    // For all other cases we can change the inmemory state to OFFLINE.
    if (timeOutMonitorReAllocate
        && (state.getState().equals(RegionState.State.PENDING_OPEN) || state
            .getState().equals(RegionState.State.OPENING))) {
      state.update(RegionState.State.PENDING_OPEN);
      allowCreation = false;
    } else {
      state.update(RegionState.State.OFFLINE);
      allowCreation = true;
    }
This change is quite tricky.  
In normal assign flow the unassigned node for the region will not be present
Hence we need to allow the creation of the node newly.
But in timeout monitor case we will have the node present in some state hence 
we decide whether to create node newly or not inside ZKAssign.createOrForceNodeOffline

The above code also updates the inmemory state of OFFLINE or PENDING_OPEN
which was not update in the previous forceRegionStateToOffline() call.
==============================================================================
In ZKAssign.java()
    if (version == -1) {
      // If timeoutmonitor deducts a node to be in OPENING state but before it
      // could
      // transit to OFFLINE state if RS had opened the region then the Master
      // deletes the
      // assigned region znode. In that case the znode will not exist. So we
      // should not
      // create the znode again which will lead to double assignment.
      if (timeOutMonitorReAllocate && !allowCreation) {
        return -1;
      }
this part prevents double assignment
If timeoutmonitor tries to force to OFFLINE state an existing region which was in RIT
but before it could do that if the node was opened then openedregionhandler will delete
the node hence we should not create the node again.

=======================================================================================
In ZkAssign.java()
    } else {
      RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
          .getEncodedName(), stat);
      // Do not move the node to OFFLINE if znode is in any of the following
      // state.
      // Because these are already executed states.
      if (timeOutMonitorReAllocate && null != curDataInZNode) {
        EventType eventType = curDataInZNode.getEventType();
        if (eventType.equals(EventType.RS_ZK_REGION_CLOSING)
            || eventType.equals(EventType.RS_ZK_REGION_CLOSED)
            || eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
          return -1;
        }
      }
This check prevents from moving the node to OFFLINE state if just before
the node is tried to force to OFFLINE the RS would have changed the state 
to either CLOSING or CLOSED or OPENED.  now again moving to OFFLINE
will lead to douoble assignment and will an additional operaition.

====================================================================================
In ZKassign.java()
      boolean setData = false;
      try {
        setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
        // Setdata throws KeeperException which aborts the Master. So we are
        // catching it here.
        // If just before setting the znode to OFFLINE if the RS has made any
        // change to the
        // znode state then we need to return -1.
      } catch (KeeperException kpe) {
        LOG.info("Version mismatch while setting the node to OFFLINE state.");
        return -1;
      }
This change is actually to avoid the master from abortng. If the forceful OFFLINE is in
progrss but just before setting the RS has changed the state.
Then the setData will fail leading to master abort.
Hence we are catching the exception.
====================================================================================

In assignmentManager.java
+      if (setOfflineInZK && versionOfOfflineNode == -1)
+        return;
This is nothing but the refactoring done in the existing code.
-     if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
So if setting the version is unsuccessful return.
=====================================================================================
In ZKassign.java()
// the below check ensure that double assignment doesnot happen.
    // When the node is created for the first time then the expected version
    // that is
    // passed will be -1 and the version in znode will be 0.
    // In all other cases the version in znode will be > 0.
    else if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
        && endState.equals(EventType.RS_ZK_REGION_OPENING)
        && expectedVersion == -1 && stat.getVersion() != 0) {
      LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
          + encoded + " from " + beginState + " to " + endState + " failed, "
          + "the node existed but was version " + stat.getVersion()
          + " not the expected version " + expectedVersion));
      return -1;
    }
As the comment explains when the node is created for first time the expectedversion will
be -1 but the actual version will be 0.  Here the scenario is
If RS1 has not tranitioned the node from OFFLINE to OPENING if
RS2 gets the call from Master after forcefully chaning to OFFLINE the
Rs2 will take the control of the node.
At that time if RS1 starts transmitting the node then we should not allow
it.
===============================================================================


This addresses bug HBASE-4015.
    https://issues.apache.org/jira/browse/HBASE-4015


Diffs (updated)
-----

  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 1162313 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java 1162313 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 1162313 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1162313 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java PRE-CREATION 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1162313 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java 1162313 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java 1162313 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java 1162313 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java 1162313 

Diff: https://reviews.apache.org/r/1668/diff


Testing
-------

Yes.  But could not add new test case.
TestMasterFailOver is passing with the current changes also.


Thanks,

ramkrishna



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Todd Lipcon (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13097009#comment-13097009 ] 

Todd Lipcon commented on HBASE-4015:
------------------------------------

Is it really impossible to write any test cases for this?

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "stack (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

stack updated HBASE-4015:
-------------------------

    Priority: Blocker  (was: Critical)

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Priority: Blocker
>             Fix For: 0.92.0
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13098828#comment-13098828 ] 

jiraposter@reviews.apache.org commented on HBASE-4015:
------------------------------------------------------


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/1668/#review1783
-----------------------------------------------------------



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4098>

    Now i have moved it into AM itself.  



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4099>

    If you see the implementation of processOpeningState it is same as the handling of the previous OPENING state handling by timeout monitor.   Just that now we call it immediately instead of timeout monitor acting on it.
    
    "Should we at least check the znode again before going ahead with assign? "
    
    Now for the possibility of the znode moving to OPENED state before we try to assign to a new RS, our new logic of mulitiple checks that we do before a node can be forcefully moved to OFFLINE will handle the scenario that you have told.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4100>

    changed the method name



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4101>

    formatting done and the variable name has been changed from reassign to hijackAndPreempt.
    
    Stack last time you asked to change the variable name to reassign :)).



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4102>

    invoke from timeout monitor has been removed everywhere.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4103>

    done



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4104>

    done



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4105>

    Before we try changing the znode state if some failure happens then it means RS succeeded.  So in this place we need not change the RS state. Correct me if am wrong 



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4106>

    Next time will not repeat.  When we refactor a piece of code to a new api and apply format on that the formatter does this.
    Sometimes it gets missed out stack. Sorry



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4107>

    This is an existing code.  just the code has been moved into a method



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
<https://reviews.apache.org/r/1668/#comment4108>

    done



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
<https://reviews.apache.org/r/1668/#comment4109>

    done



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
<https://reviews.apache.org/r/1668/#comment4110>

    done



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
<https://reviews.apache.org/r/1668/#comment4111>

    done



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
<https://reviews.apache.org/r/1668/#comment4112>

    done



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
<https://reviews.apache.org/r/1668/#comment4113>

    done



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
<https://reviews.apache.org/r/1668/#comment4114>

    This statement will increment the version number.  But the stat object is got before this step.
    hence on successful creation of the node with offline state we create a new watcher and so the version+1 will give me the current exact version in the node.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
<https://reviews.apache.org/r/1668/#comment4115>

    done


- ramkrishna


On 2011-08-29 12:11:23, ramkrishna vasudevan wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/1668/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2011-08-29 12:11:23)
bq.  
bq.  
bq.  Review request for Ted Yu, Michael Stack, Jean-Daniel Cryans, and Jonathan Gray.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  HBASE-4015 - updated patch.
bq.  
bq.  
bq.          invokeTimeOutManager(regionState.getRegion(),
bq.              TimeOutOperationType.ASSIGN);
bq.  
bq.  Instead of storing the state when timeout was deducted we process it
bq.  using future task.
bq.  ================================================================
bq.  
bq.        case RS_ZK_REGION_OPENING:
bq.          // TODO: Could check if it was on deadServers.  If it was, then we could
bq.          // do what happens in TimeoutMonitor when it sees this condition.
bq.  
bq.          // Just insert region into RIT
bq.          // If this never updates the timeout will trigger new assignment
bq.          if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
bq.            regionsInTransition.put(encodedRegionName, new RegionState(
bq.                regionInfo, RegionState.State.OPENING, data.getStamp(), data
bq.                    .getOrigin()));
bq.            processOpeningState(regionInfo);
bq.            break;
bq.          }
bq.          regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
bq.              RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
bq.          break;
bq.  
bq.  This change is for HBASE-4203. META and ROOT table need not wait till timeout
bq.  ======================================================================
bq.  
bq.  In forceRegionStateToOffline()
bq.  
bq.      } else {
bq.        // If invoked from timeout monitor donot force it to OFFLINE. Based on the
bq.        // state we will decide if to change in-memory state to OFFLINE or not.  It will
bq.        // be done before setting the znode to OFFLINE state.
bq.        if (!timeOutMonitorReAllocate) {
bq.          LOG.debug("Forcing OFFLINE; was=" + state);
bq.          state.update(RegionState.State.OFFLINE);
bq.        }
bq.  If the timeout monitor tries to reallcoate the node then dont make
bq.  the inmemory state to OFFLINE.
bq.  But the noraml assign flow doesnot expect the inmemory state to OFFLINE.
bq.  Hence the above change.  This is continued with the check in 
bq.  assign()
bq.  int setOfflineInZooKeeper(final RegionState state,
bq.        boolean timeOutMonitorReAllocate) {
bq.      // If invoked from timeoutmonitor the current state in memory need not be
bq.      // OFFLINE.  
bq.      if (!timeOutMonitorReAllocate && !state.isClosed() && !state.isOffline()) {
bq.            this.master.abort("Unexpected state trying to OFFLINE; " + state,
bq.            new IllegalStateException());
bq.        return -1;
bq.      }
bq.  ======================================================================
bq.  
bq.      boolean allowCreation = false;
bq.      // If the isReAllocate is true and the current state is PENDING_OPEN
bq.      // or OPENING then update the inmemory state to PENDING_OPEN. This is
bq.      // important because
bq.      // if timeoutmonitor deducts that a region was in OPENING state for a long
bq.      // time but by the
bq.      // time timeout monitor tranits the node to OFFLINE the RS would have opened
bq.      // the node and the
bq.      // state in znode will be RS_ZK_REGION_OPENED. Inorder to invoke the
bq.      // OpenedRegionHandler
bq.      // we expect the inmemeory state to be PENDING_OPEN or OPENING.
bq.      // For all other cases we can change the inmemory state to OFFLINE.
bq.      if (timeOutMonitorReAllocate
bq.          && (state.getState().equals(RegionState.State.PENDING_OPEN) || state
bq.              .getState().equals(RegionState.State.OPENING))) {
bq.        state.update(RegionState.State.PENDING_OPEN);
bq.        allowCreation = false;
bq.      } else {
bq.        state.update(RegionState.State.OFFLINE);
bq.        allowCreation = true;
bq.      }
bq.  This change is quite tricky.  
bq.  In normal assign flow the unassigned node for the region will not be present
bq.  Hence we need to allow the creation of the node newly.
bq.  But in timeout monitor case we will have the node present in some state hence 
bq.  we decide whether to create node newly or not inside ZKAssign.createOrForceNodeOffline
bq.  
bq.  The above code also updates the inmemory state of OFFLINE or PENDING_OPEN
bq.  which was not update in the previous forceRegionStateToOffline() call.
bq.  ==============================================================================
bq.  In ZKAssign.java()
bq.      if (version == -1) {
bq.        // If timeoutmonitor deducts a node to be in OPENING state but before it
bq.        // could
bq.        // transit to OFFLINE state if RS had opened the region then the Master
bq.        // deletes the
bq.        // assigned region znode. In that case the znode will not exist. So we
bq.        // should not
bq.        // create the znode again which will lead to double assignment.
bq.        if (timeOutMonitorReAllocate && !allowCreation) {
bq.          return -1;
bq.        }
bq.  this part prevents double assignment
bq.  If timeoutmonitor tries to force to OFFLINE state an existing region which was in RIT
bq.  but before it could do that if the node was opened then openedregionhandler will delete
bq.  the node hence we should not create the node again.
bq.  
bq.  =======================================================================================
bq.  In ZkAssign.java()
bq.      } else {
bq.        RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
bq.            .getEncodedName(), stat);
bq.        // Do not move the node to OFFLINE if znode is in any of the following
bq.        // state.
bq.        // Because these are already executed states.
bq.        if (timeOutMonitorReAllocate && null != curDataInZNode) {
bq.          EventType eventType = curDataInZNode.getEventType();
bq.          if (eventType.equals(EventType.RS_ZK_REGION_CLOSING)
bq.              || eventType.equals(EventType.RS_ZK_REGION_CLOSED)
bq.              || eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
bq.            return -1;
bq.          }
bq.        }
bq.  This check prevents from moving the node to OFFLINE state if just before
bq.  the node is tried to force to OFFLINE the RS would have changed the state 
bq.  to either CLOSING or CLOSED or OPENED.  now again moving to OFFLINE
bq.  will lead to douoble assignment and will an additional operaition.
bq.  
bq.  ====================================================================================
bq.  In ZKassign.java()
bq.        boolean setData = false;
bq.        try {
bq.          setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
bq.          // Setdata throws KeeperException which aborts the Master. So we are
bq.          // catching it here.
bq.          // If just before setting the znode to OFFLINE if the RS has made any
bq.          // change to the
bq.          // znode state then we need to return -1.
bq.        } catch (KeeperException kpe) {
bq.          LOG.info("Version mismatch while setting the node to OFFLINE state.");
bq.          return -1;
bq.        }
bq.  This change is actually to avoid the master from abortng. If the forceful OFFLINE is in
bq.  progrss but just before setting the RS has changed the state.
bq.  Then the setData will fail leading to master abort.
bq.  Hence we are catching the exception.
bq.  ====================================================================================
bq.  
bq.  In assignmentManager.java
bq.  +      if (setOfflineInZK && versionOfOfflineNode == -1)
bq.  +        return;
bq.  This is nothing but the refactoring done in the existing code.
bq.  -     if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
bq.  So if setting the version is unsuccessful return.
bq.  =====================================================================================
bq.  In ZKassign.java()
bq.  // the below check ensure that double assignment doesnot happen.
bq.      // When the node is created for the first time then the expected version
bq.      // that is
bq.      // passed will be -1 and the version in znode will be 0.
bq.      // In all other cases the version in znode will be > 0.
bq.      else if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
bq.          && endState.equals(EventType.RS_ZK_REGION_OPENING)
bq.          && expectedVersion == -1 && stat.getVersion() != 0) {
bq.        LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
bq.            + encoded + " from " + beginState + " to " + endState + " failed, "
bq.            + "the node existed but was version " + stat.getVersion()
bq.            + " not the expected version " + expectedVersion));
bq.        return -1;
bq.      }
bq.  As the comment explains when the node is created for first time the expectedversion will
bq.  be -1 but the actual version will be 0.  Here the scenario is
bq.  If RS1 has not tranitioned the node from OFFLINE to OPENING if
bq.  RS2 gets the call from Master after forcefully chaning to OFFLINE the
bq.  Rs2 will take the control of the node.
bq.  At that time if RS1 starts transmitting the node then we should not allow
bq.  it.
bq.  ===============================================================================
bq.  
bq.  
bq.  This addresses bug HBASE-4015.
bq.      https://issues.apache.org/jira/browse/HBASE-4015
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java PRE-CREATION 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java 1161985 
bq.  
bq.  Diff: https://reviews.apache.org/r/1668/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  Yes.  But could not add new test case.
bq.  TestMasterFailOver is passing with the current changes also.
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  ramkrishna
bq.  
bq.



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13097006#comment-13097006 ] 

Ted Yu commented on HBASE-4015:
-------------------------------

+1 on patch.
Nice work Ramkrishna.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

ramkrishna.s.vasudevan updated HBASE-4015:
------------------------------------------

    Status: Patch Available  (was: Open)

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13097144#comment-13097144 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

I tried to write testcases to simulate the scenario for timeout monitor,
Based on the state in RIT we need to decide the operations.  So I tried like before assigning some regions start a thread that has gets the reference to assignement manager and gets the list of RITS(checking the state of RIT in a loop). Based on the state is PENDING_OPEN or OPENING try changing the state in znode to CLOSING.
Now when we try to check the status there is no guarentee that the moment we get the status and when the RS will process it to OPENED.

Trying to do something similar to TestMasterFailOver.  But in TestMasterFailOver the Master is aborted and then we do the changes to the znode.
But when the master is running, the operation of assigning regions is asynchronous and we cannot gurantee reliability.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13099230#comment-13099230 ] 

jiraposter@reviews.apache.org commented on HBASE-4015:
------------------------------------------------------


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/1668/#review1800
-----------------------------------------------------------


This patch is hard to review as it's very shotgun surgery-y, but that's probably because of the nature of the required change.

I'd like to know more about the testing that was done. How many regions on how many nodes?

- Jean-Daniel


On 2011-09-07 15:52:26, ramkrishna vasudevan wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/1668/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2011-09-07 15:52:26)
bq.  
bq.  
bq.  Review request for Ted Yu, Michael Stack, Jean-Daniel Cryans, and Jonathan Gray.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  HBASE-4015 - updated patch.
bq.  
bq.  
bq.          invokeTimeOutManager(regionState.getRegion(),
bq.              TimeOutOperationType.ASSIGN);
bq.  
bq.  Instead of storing the state when timeout was deducted we process it
bq.  using future task.
bq.  ================================================================
bq.  
bq.        case RS_ZK_REGION_OPENING:
bq.          // TODO: Could check if it was on deadServers.  If it was, then we could
bq.          // do what happens in TimeoutMonitor when it sees this condition.
bq.  
bq.          // Just insert region into RIT
bq.          // If this never updates the timeout will trigger new assignment
bq.          if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
bq.            regionsInTransition.put(encodedRegionName, new RegionState(
bq.                regionInfo, RegionState.State.OPENING, data.getStamp(), data
bq.                    .getOrigin()));
bq.            processOpeningState(regionInfo);
bq.            break;
bq.          }
bq.          regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
bq.              RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
bq.          break;
bq.  
bq.  This change is for HBASE-4203. META and ROOT table need not wait till timeout
bq.  ======================================================================
bq.  
bq.  In forceRegionStateToOffline()
bq.  
bq.      } else {
bq.        // If invoked from timeout monitor donot force it to OFFLINE. Based on the
bq.        // state we will decide if to change in-memory state to OFFLINE or not.  It will
bq.        // be done before setting the znode to OFFLINE state.
bq.        if (!timeOutMonitorReAllocate) {
bq.          LOG.debug("Forcing OFFLINE; was=" + state);
bq.          state.update(RegionState.State.OFFLINE);
bq.        }
bq.  If the timeout monitor tries to reallcoate the node then dont make
bq.  the inmemory state to OFFLINE.
bq.  But the noraml assign flow doesnot expect the inmemory state to OFFLINE.
bq.  Hence the above change.  This is continued with the check in 
bq.  assign()
bq.  int setOfflineInZooKeeper(final RegionState state,
bq.        boolean timeOutMonitorReAllocate) {
bq.      // If invoked from timeoutmonitor the current state in memory need not be
bq.      // OFFLINE.  
bq.      if (!timeOutMonitorReAllocate && !state.isClosed() && !state.isOffline()) {
bq.            this.master.abort("Unexpected state trying to OFFLINE; " + state,
bq.            new IllegalStateException());
bq.        return -1;
bq.      }
bq.  ======================================================================
bq.  
bq.      boolean allowCreation = false;
bq.      // If the isReAllocate is true and the current state is PENDING_OPEN
bq.      // or OPENING then update the inmemory state to PENDING_OPEN. This is
bq.      // important because
bq.      // if timeoutmonitor deducts that a region was in OPENING state for a long
bq.      // time but by the
bq.      // time timeout monitor tranits the node to OFFLINE the RS would have opened
bq.      // the node and the
bq.      // state in znode will be RS_ZK_REGION_OPENED. Inorder to invoke the
bq.      // OpenedRegionHandler
bq.      // we expect the inmemeory state to be PENDING_OPEN or OPENING.
bq.      // For all other cases we can change the inmemory state to OFFLINE.
bq.      if (timeOutMonitorReAllocate
bq.          && (state.getState().equals(RegionState.State.PENDING_OPEN) || state
bq.              .getState().equals(RegionState.State.OPENING))) {
bq.        state.update(RegionState.State.PENDING_OPEN);
bq.        allowCreation = false;
bq.      } else {
bq.        state.update(RegionState.State.OFFLINE);
bq.        allowCreation = true;
bq.      }
bq.  This change is quite tricky.  
bq.  In normal assign flow the unassigned node for the region will not be present
bq.  Hence we need to allow the creation of the node newly.
bq.  But in timeout monitor case we will have the node present in some state hence 
bq.  we decide whether to create node newly or not inside ZKAssign.createOrForceNodeOffline
bq.  
bq.  The above code also updates the inmemory state of OFFLINE or PENDING_OPEN
bq.  which was not update in the previous forceRegionStateToOffline() call.
bq.  ==============================================================================
bq.  In ZKAssign.java()
bq.      if (version == -1) {
bq.        // If timeoutmonitor deducts a node to be in OPENING state but before it
bq.        // could
bq.        // transit to OFFLINE state if RS had opened the region then the Master
bq.        // deletes the
bq.        // assigned region znode. In that case the znode will not exist. So we
bq.        // should not
bq.        // create the znode again which will lead to double assignment.
bq.        if (timeOutMonitorReAllocate && !allowCreation) {
bq.          return -1;
bq.        }
bq.  this part prevents double assignment
bq.  If timeoutmonitor tries to force to OFFLINE state an existing region which was in RIT
bq.  but before it could do that if the node was opened then openedregionhandler will delete
bq.  the node hence we should not create the node again.
bq.  
bq.  =======================================================================================
bq.  In ZkAssign.java()
bq.      } else {
bq.        RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
bq.            .getEncodedName(), stat);
bq.        // Do not move the node to OFFLINE if znode is in any of the following
bq.        // state.
bq.        // Because these are already executed states.
bq.        if (timeOutMonitorReAllocate && null != curDataInZNode) {
bq.          EventType eventType = curDataInZNode.getEventType();
bq.          if (eventType.equals(EventType.RS_ZK_REGION_CLOSING)
bq.              || eventType.equals(EventType.RS_ZK_REGION_CLOSED)
bq.              || eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
bq.            return -1;
bq.          }
bq.        }
bq.  This check prevents from moving the node to OFFLINE state if just before
bq.  the node is tried to force to OFFLINE the RS would have changed the state 
bq.  to either CLOSING or CLOSED or OPENED.  now again moving to OFFLINE
bq.  will lead to douoble assignment and will an additional operaition.
bq.  
bq.  ====================================================================================
bq.  In ZKassign.java()
bq.        boolean setData = false;
bq.        try {
bq.          setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
bq.          // Setdata throws KeeperException which aborts the Master. So we are
bq.          // catching it here.
bq.          // If just before setting the znode to OFFLINE if the RS has made any
bq.          // change to the
bq.          // znode state then we need to return -1.
bq.        } catch (KeeperException kpe) {
bq.          LOG.info("Version mismatch while setting the node to OFFLINE state.");
bq.          return -1;
bq.        }
bq.  This change is actually to avoid the master from abortng. If the forceful OFFLINE is in
bq.  progrss but just before setting the RS has changed the state.
bq.  Then the setData will fail leading to master abort.
bq.  Hence we are catching the exception.
bq.  ====================================================================================
bq.  
bq.  In assignmentManager.java
bq.  +      if (setOfflineInZK && versionOfOfflineNode == -1)
bq.  +        return;
bq.  This is nothing but the refactoring done in the existing code.
bq.  -     if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
bq.  So if setting the version is unsuccessful return.
bq.  =====================================================================================
bq.  In ZKassign.java()
bq.  // the below check ensure that double assignment doesnot happen.
bq.      // When the node is created for the first time then the expected version
bq.      // that is
bq.      // passed will be -1 and the version in znode will be 0.
bq.      // In all other cases the version in znode will be > 0.
bq.      else if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
bq.          && endState.equals(EventType.RS_ZK_REGION_OPENING)
bq.          && expectedVersion == -1 && stat.getVersion() != 0) {
bq.        LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
bq.            + encoded + " from " + beginState + " to " + endState + " failed, "
bq.            + "the node existed but was version " + stat.getVersion()
bq.            + " not the expected version " + expectedVersion));
bq.        return -1;
bq.      }
bq.  As the comment explains when the node is created for first time the expectedversion will
bq.  be -1 but the actual version will be 0.  Here the scenario is
bq.  If RS1 has not tranitioned the node from OFFLINE to OPENING if
bq.  RS2 gets the call from Master after forcefully chaning to OFFLINE the
bq.  Rs2 will take the control of the node.
bq.  At that time if RS1 starts transmitting the node then we should not allow
bq.  it.
bq.  ===============================================================================
bq.  
bq.  
bq.  This addresses bug HBASE-4015.
bq.      https://issues.apache.org/jira/browse/HBASE-4015
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 1166224 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignCallable.java PRE-CREATION 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java 1166224 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 1166224 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1166224 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/UnAssignCallable.java PRE-CREATION 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1166224 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java 1166224 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java 1166224 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java 1166224 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java 1166224 
bq.  
bq.  Diff: https://reviews.apache.org/r/1668/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  Yes.  But could not add new test case.
bq.  TestMasterFailOver is passing with the current changes also.
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  ramkrishna
bq.  
bq.



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13100367#comment-13100367 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

@J-D

bq.You could also try doing a worst case cold startup by killing -9 all HBase components at the same time (more or less) and then restarting them all (also after data was added). Finally you could try setting a super low timeout setting, like 5 seconds, to trigger RIT timeouts by the hundreds.

I conducted the tests again particularly with 5 secs time out. Killed the cluster, started again, Randomly killed RS -> invoked balancer command also.
I was able to get back all the regions (4003 regions) among 3 RS.
hbck result was also positive
{noformat}
***** The number of timed out regions **** 938
***** The number of timed out regions **** 270
***** The number of timed out regions **** 673
***** The number of timed out regions **** 269
***** The number of timed out regions **** 941
***** The number of timed out regions **** 942
***** The number of timed out regions **** 941
{noformat}

{noformat}
Summary:
  -ROOT- is okay.
    Number of regions: 1
    Deployed on:  HOST-10-18-52-253,60020,1315480076091
  .META. is okay.
    Number of regions: 1
    Deployed on:  HOST-10-18-52-253,60020,1315480076091
  testram2 is okay.
    Number of regions: 4001
    Deployed on:  HOST-10-18-52-108,60020,1315480229321 HOST-10-18-52-253,60020,1315480076091
0 inconsistencies detected.
Status: OK
{noformat}

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13072166#comment-13072166 ] 

stack commented on HBASE-4015:
------------------------------

No.  Too important.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Priority: Critical
>             Fix For: 0.92.0
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Assigned] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

ramkrishna.s.vasudevan reassigned HBASE-4015:
---------------------------------------------

    Assignee: ramkrishna.s.vasudevan

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

ramkrishna.s.vasudevan updated HBASE-4015:
------------------------------------------

    Attachment: HBASE-4015_reprepared_trunk_2.patch

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13080036#comment-13080036 ] 

Ted Yu commented on HBASE-4015:
-------------------------------

I would choose total number of region servers to be the limit beyond which master stops assigning the same region.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

ramkrishna.s.vasudevan updated HBASE-4015:
------------------------------------------

    Status: Patch Available  (was: Open)

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13090069#comment-13090069 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

@Ted
Fine.. i will do that.  The patch has become outdated.
I will update and send it to review board.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13099266#comment-13099266 ] 

Ted Yu commented on HBASE-4015:
-------------------------------

@J-D:
See Ramkrishna's comment @ 02/Sep/11 16:56

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13082242#comment-13082242 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

@Stack, 
{noformat}
Do we need new RS_ALLOCATE state? Could we just have OFFLINE plus your suggestion of adding RS name so its OFFLINE+RS_TO_OPEN_REGION_ON?
{noformat}
Yes this may also be possible.  But we thought of introducing a new state so that there is a clear distinction whether reallocation has happened or not and also handling of the new state may be cleaner than changing the behaviour in the existing state.

{noformat}
What happens if we assign the region back to RS1 (it can happen).
{noformat}
Yes.  we have considered this scenario also.  If the region is reallocated to the same RS there are two flows

-> If the state is OPENING in zk but it is still not added to online regions list in RS then any subsequent call from MASTER to RS with RE_ALLOCATE state will succeed but the previous processing from OPENING to OPEN will fail.
-> In the second case if the region is added to the online regions list then the RS will say ALREADY_OPENED and before removing from RIT in master we will check if the node is deleted if not it will not be removed from RIT. Hence the state will be in  PENDING_OPEN so subsequent timeout monitor call will handle it.

Pls provide your suggestions.


> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

ramkrishna.s.vasudevan updated HBASE-4015:
------------------------------------------

    Status: Open  (was: Patch Available)

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13087759#comment-13087759 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

@Stack
I have uploaded the patch with soln as discussed.  The OFFLINE state is used now instead of new state.

But Stack, from my side I feel the prev implementation was much easier and cleaner.  Pls check both the patches and provide your comments.  Thanks.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13080407#comment-13080407 ] 

Ted Yu commented on HBASE-4015:
-------------------------------

The verification of state is what I was thinking.
Good job Ramkrishna.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13085527#comment-13085527 ] 

stack commented on HBASE-4015:
------------------------------

@Jon I took a look at passing the seqno and its really invasive -- as written, writing, its hard to get the seqno -- and then passing it over rpc to RS and then down into the executor that does the open would require changing all method sigs and RS interfaces.  That said, it would make for the most robust implementation; for sure we'd not need a new state.

@Ram Perhaps we can work on this together?  You on for doing the passing of the seqno?  I'd like to help.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13092283#comment-13092283 ] 

jiraposter@reviews.apache.org commented on HBASE-4015:
------------------------------------------------------



bq.  On 2011-08-27 02:55:47, Ted Yu wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 2362
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line2362>
bq.  >
bq.  >     The two strings can be concatenated:
bq.  >      region close",

done


- ramkrishna


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/1668/#review1668
-----------------------------------------------------------


On 2011-08-27 11:51:16, ramkrishna vasudevan wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/1668/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2011-08-27 11:51:16)
bq.  
bq.  
bq.  Review request for Ted Yu, Michael Stack, Jean-Daniel Cryans, and Jonathan Gray.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  HBASE-4015 - updated patch.
bq.  
bq.  
bq.          invokeTimeOutManager(regionState.getRegion(),
bq.              TimeOutOperationType.ASSIGN);
bq.  
bq.  Instead of storing the state when timeout was deducted we process it
bq.  using future task.
bq.  ================================================================
bq.  
bq.        case RS_ZK_REGION_OPENING:
bq.          // TODO: Could check if it was on deadServers.  If it was, then we could
bq.          // do what happens in TimeoutMonitor when it sees this condition.
bq.  
bq.          // Just insert region into RIT
bq.          // If this never updates the timeout will trigger new assignment
bq.          if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
bq.            regionsInTransition.put(encodedRegionName, new RegionState(
bq.                regionInfo, RegionState.State.OPENING, data.getStamp(), data
bq.                    .getOrigin()));
bq.            processOpeningState(regionInfo);
bq.            break;
bq.          }
bq.          regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
bq.              RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
bq.          break;
bq.  
bq.  This change is for HBASE-4203. META and ROOT table need not wait till timeout
bq.  ======================================================================
bq.  
bq.  In forceRegionStateToOffline()
bq.  
bq.      } else {
bq.        // If invoked from timeout monitor donot force it to OFFLINE. Based on the
bq.        // state we will decide if to change in-memory state to OFFLINE or not.  It will
bq.        // be done before setting the znode to OFFLINE state.
bq.        if (!timeOutMonitorReAllocate) {
bq.          LOG.debug("Forcing OFFLINE; was=" + state);
bq.          state.update(RegionState.State.OFFLINE);
bq.        }
bq.  If the timeout monitor tries to reallcoate the node then dont make
bq.  the inmemory state to OFFLINE.
bq.  But the noraml assign flow doesnot expect the inmemory state to OFFLINE.
bq.  Hence the above change.  This is continued with the check in 
bq.  assign()
bq.  int setOfflineInZooKeeper(final RegionState state,
bq.        boolean timeOutMonitorReAllocate) {
bq.      // If invoked from timeoutmonitor the current state in memory need not be
bq.      // OFFLINE.  
bq.      if (!timeOutMonitorReAllocate && !state.isClosed() && !state.isOffline()) {
bq.            this.master.abort("Unexpected state trying to OFFLINE; " + state,
bq.            new IllegalStateException());
bq.        return -1;
bq.      }
bq.  ======================================================================
bq.  
bq.      boolean allowCreation = false;
bq.      // If the isReAllocate is true and the current state is PENDING_OPEN
bq.      // or OPENING then update the inmemory state to PENDING_OPEN. This is
bq.      // important because
bq.      // if timeoutmonitor deducts that a region was in OPENING state for a long
bq.      // time but by the
bq.      // time timeout monitor tranits the node to OFFLINE the RS would have opened
bq.      // the node and the
bq.      // state in znode will be RS_ZK_REGION_OPENED. Inorder to invoke the
bq.      // OpenedRegionHandler
bq.      // we expect the inmemeory state to be PENDING_OPEN or OPENING.
bq.      // For all other cases we can change the inmemory state to OFFLINE.
bq.      if (timeOutMonitorReAllocate
bq.          && (state.getState().equals(RegionState.State.PENDING_OPEN) || state
bq.              .getState().equals(RegionState.State.OPENING))) {
bq.        state.update(RegionState.State.PENDING_OPEN);
bq.        allowCreation = false;
bq.      } else {
bq.        state.update(RegionState.State.OFFLINE);
bq.        allowCreation = true;
bq.      }
bq.  This change is quite tricky.  
bq.  In normal assign flow the unassigned node for the region will not be present
bq.  Hence we need to allow the creation of the node newly.
bq.  But in timeout monitor case we will have the node present in some state hence 
bq.  we decide whether to create node newly or not inside ZKAssign.createOrForceNodeOffline
bq.  
bq.  The above code also updates the inmemory state of OFFLINE or PENDING_OPEN
bq.  which was not update in the previous forceRegionStateToOffline() call.
bq.  ==============================================================================
bq.  In ZKAssign.java()
bq.      if (version == -1) {
bq.        // If timeoutmonitor deducts a node to be in OPENING state but before it
bq.        // could
bq.        // transit to OFFLINE state if RS had opened the region then the Master
bq.        // deletes the
bq.        // assigned region znode. In that case the znode will not exist. So we
bq.        // should not
bq.        // create the znode again which will lead to double assignment.
bq.        if (timeOutMonitorReAllocate && !allowCreation) {
bq.          return -1;
bq.        }
bq.  this part prevents double assignment
bq.  If timeoutmonitor tries to force to OFFLINE state an existing region which was in RIT
bq.  but before it could do that if the node was opened then openedregionhandler will delete
bq.  the node hence we should not create the node again.
bq.  
bq.  =======================================================================================
bq.  In ZkAssign.java()
bq.      } else {
bq.        RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
bq.            .getEncodedName(), stat);
bq.        // Do not move the node to OFFLINE if znode is in any of the following
bq.        // state.
bq.        // Because these are already executed states.
bq.        if (timeOutMonitorReAllocate && null != curDataInZNode) {
bq.          EventType eventType = curDataInZNode.getEventType();
bq.          if (eventType.equals(EventType.RS_ZK_REGION_CLOSING)
bq.              || eventType.equals(EventType.RS_ZK_REGION_CLOSED)
bq.              || eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
bq.            return -1;
bq.          }
bq.        }
bq.  This check prevents from moving the node to OFFLINE state if just before
bq.  the node is tried to force to OFFLINE the RS would have changed the state 
bq.  to either CLOSING or CLOSED or OPENED.  now again moving to OFFLINE
bq.  will lead to douoble assignment and will an additional operaition.
bq.  
bq.  ====================================================================================
bq.  In ZKassign.java()
bq.        boolean setData = false;
bq.        try {
bq.          setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
bq.          // Setdata throws KeeperException which aborts the Master. So we are
bq.          // catching it here.
bq.          // If just before setting the znode to OFFLINE if the RS has made any
bq.          // change to the
bq.          // znode state then we need to return -1.
bq.        } catch (KeeperException kpe) {
bq.          LOG.info("Version mismatch while setting the node to OFFLINE state.");
bq.          return -1;
bq.        }
bq.  This change is actually to avoid the master from abortng. If the forceful OFFLINE is in
bq.  progrss but just before setting the RS has changed the state.
bq.  Then the setData will fail leading to master abort.
bq.  Hence we are catching the exception.
bq.  ====================================================================================
bq.  
bq.  In assignmentManager.java
bq.  +      if (setOfflineInZK && versionOfOfflineNode == -1)
bq.  +        return;
bq.  This is nothing but the refactoring done in the existing code.
bq.  -     if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
bq.  So if setting the version is unsuccessful return.
bq.  =====================================================================================
bq.  In ZKassign.java()
bq.  // the below check ensure that double assignment doesnot happen.
bq.      // When the node is created for the first time then the expected version
bq.      // that is
bq.      // passed will be -1 and the version in znode will be 0.
bq.      // In all other cases the version in znode will be > 0.
bq.      else if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
bq.          && endState.equals(EventType.RS_ZK_REGION_OPENING)
bq.          && expectedVersion == -1 && stat.getVersion() != 0) {
bq.        LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
bq.            + encoded + " from " + beginState + " to " + endState + " failed, "
bq.            + "the node existed but was version " + stat.getVersion()
bq.            + " not the expected version " + expectedVersion));
bq.        return -1;
bq.      }
bq.  As the comment explains when the node is created for first time the expectedversion will
bq.  be -1 but the actual version will be 0.  Here the scenario is
bq.  If RS1 has not tranitioned the node from OFFLINE to OPENING if
bq.  RS2 gets the call from Master after forcefully chaning to OFFLINE the
bq.  Rs2 will take the control of the node.
bq.  At that time if RS1 starts transmitting the node then we should not allow
bq.  it.
bq.  ===============================================================================
bq.  
bq.  
bq.  This addresses bug HBASE-4015.
bq.      https://issues.apache.org/jira/browse/HBASE-4015
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java PRE-CREATION 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java 1162313 
bq.  
bq.  Diff: https://reviews.apache.org/r/1668/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  Yes.  But could not add new test case.
bq.  TestMasterFailOver is passing with the current changes also.
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  ramkrishna
bq.  
bq.



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13100645#comment-13100645 ] 

Hudson commented on HBASE-4015:
-------------------------------

Integrated in HBase-TRUNK #2189 (See [https://builds.apache.org/job/HBase-TRUNK/2189/])
    HBASE-4015 Refactor the TimeoutMonitor to make it less racy
HBASE-4015 Refactor the TimeoutMonitor to make it less racy
HBASE-4015 Refactor the TimeoutMonitor to make it less racy

stack : 
Files : 
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java

stack : 
Files : 
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java

stack : 
Files : 
* /hbase/trunk/CHANGES.txt
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java
* /hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java


> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13092793#comment-13092793 ] 

jiraposter@reviews.apache.org commented on HBASE-4015:
------------------------------------------------------


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/1668/
-----------------------------------------------------------

(Updated 2011-08-29 12:11:23.781032)


Review request for Ted Yu, Michael Stack, Jean-Daniel Cryans, and Jonathan Gray.


Summary
-------

HBASE-4015 - updated patch.


        invokeTimeOutManager(regionState.getRegion(),
            TimeOutOperationType.ASSIGN);

Instead of storing the state when timeout was deducted we process it
using future task.
================================================================

      case RS_ZK_REGION_OPENING:
        // TODO: Could check if it was on deadServers.  If it was, then we could
        // do what happens in TimeoutMonitor when it sees this condition.

        // Just insert region into RIT
        // If this never updates the timeout will trigger new assignment
        if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
          regionsInTransition.put(encodedRegionName, new RegionState(
              regionInfo, RegionState.State.OPENING, data.getStamp(), data
                  .getOrigin()));
          processOpeningState(regionInfo);
          break;
        }
        regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
            RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
        break;

This change is for HBASE-4203. META and ROOT table need not wait till timeout
======================================================================

In forceRegionStateToOffline()

    } else {
      // If invoked from timeout monitor donot force it to OFFLINE. Based on the
      // state we will decide if to change in-memory state to OFFLINE or not.  It will
      // be done before setting the znode to OFFLINE state.
      if (!timeOutMonitorReAllocate) {
        LOG.debug("Forcing OFFLINE; was=" + state);
        state.update(RegionState.State.OFFLINE);
      }
If the timeout monitor tries to reallcoate the node then dont make
the inmemory state to OFFLINE.
But the noraml assign flow doesnot expect the inmemory state to OFFLINE.
Hence the above change.  This is continued with the check in 
assign()
int setOfflineInZooKeeper(final RegionState state,
      boolean timeOutMonitorReAllocate) {
    // If invoked from timeoutmonitor the current state in memory need not be
    // OFFLINE.  
    if (!timeOutMonitorReAllocate && !state.isClosed() && !state.isOffline()) {
          this.master.abort("Unexpected state trying to OFFLINE; " + state,
          new IllegalStateException());
      return -1;
    }
======================================================================

    boolean allowCreation = false;
    // If the isReAllocate is true and the current state is PENDING_OPEN
    // or OPENING then update the inmemory state to PENDING_OPEN. This is
    // important because
    // if timeoutmonitor deducts that a region was in OPENING state for a long
    // time but by the
    // time timeout monitor tranits the node to OFFLINE the RS would have opened
    // the node and the
    // state in znode will be RS_ZK_REGION_OPENED. Inorder to invoke the
    // OpenedRegionHandler
    // we expect the inmemeory state to be PENDING_OPEN or OPENING.
    // For all other cases we can change the inmemory state to OFFLINE.
    if (timeOutMonitorReAllocate
        && (state.getState().equals(RegionState.State.PENDING_OPEN) || state
            .getState().equals(RegionState.State.OPENING))) {
      state.update(RegionState.State.PENDING_OPEN);
      allowCreation = false;
    } else {
      state.update(RegionState.State.OFFLINE);
      allowCreation = true;
    }
This change is quite tricky.  
In normal assign flow the unassigned node for the region will not be present
Hence we need to allow the creation of the node newly.
But in timeout monitor case we will have the node present in some state hence 
we decide whether to create node newly or not inside ZKAssign.createOrForceNodeOffline

The above code also updates the inmemory state of OFFLINE or PENDING_OPEN
which was not update in the previous forceRegionStateToOffline() call.
==============================================================================
In ZKAssign.java()
    if (version == -1) {
      // If timeoutmonitor deducts a node to be in OPENING state but before it
      // could
      // transit to OFFLINE state if RS had opened the region then the Master
      // deletes the
      // assigned region znode. In that case the znode will not exist. So we
      // should not
      // create the znode again which will lead to double assignment.
      if (timeOutMonitorReAllocate && !allowCreation) {
        return -1;
      }
this part prevents double assignment
If timeoutmonitor tries to force to OFFLINE state an existing region which was in RIT
but before it could do that if the node was opened then openedregionhandler will delete
the node hence we should not create the node again.

=======================================================================================
In ZkAssign.java()
    } else {
      RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
          .getEncodedName(), stat);
      // Do not move the node to OFFLINE if znode is in any of the following
      // state.
      // Because these are already executed states.
      if (timeOutMonitorReAllocate && null != curDataInZNode) {
        EventType eventType = curDataInZNode.getEventType();
        if (eventType.equals(EventType.RS_ZK_REGION_CLOSING)
            || eventType.equals(EventType.RS_ZK_REGION_CLOSED)
            || eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
          return -1;
        }
      }
This check prevents from moving the node to OFFLINE state if just before
the node is tried to force to OFFLINE the RS would have changed the state 
to either CLOSING or CLOSED or OPENED.  now again moving to OFFLINE
will lead to douoble assignment and will an additional operaition.

====================================================================================
In ZKassign.java()
      boolean setData = false;
      try {
        setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
        // Setdata throws KeeperException which aborts the Master. So we are
        // catching it here.
        // If just before setting the znode to OFFLINE if the RS has made any
        // change to the
        // znode state then we need to return -1.
      } catch (KeeperException kpe) {
        LOG.info("Version mismatch while setting the node to OFFLINE state.");
        return -1;
      }
This change is actually to avoid the master from abortng. If the forceful OFFLINE is in
progrss but just before setting the RS has changed the state.
Then the setData will fail leading to master abort.
Hence we are catching the exception.
====================================================================================

In assignmentManager.java
+      if (setOfflineInZK && versionOfOfflineNode == -1)
+        return;
This is nothing but the refactoring done in the existing code.
-     if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
So if setting the version is unsuccessful return.
=====================================================================================
In ZKassign.java()
// the below check ensure that double assignment doesnot happen.
    // When the node is created for the first time then the expected version
    // that is
    // passed will be -1 and the version in znode will be 0.
    // In all other cases the version in znode will be > 0.
    else if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
        && endState.equals(EventType.RS_ZK_REGION_OPENING)
        && expectedVersion == -1 && stat.getVersion() != 0) {
      LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
          + encoded + " from " + beginState + " to " + endState + " failed, "
          + "the node existed but was version " + stat.getVersion()
          + " not the expected version " + expectedVersion));
      return -1;
    }
As the comment explains when the node is created for first time the expectedversion will
be -1 but the actual version will be 0.  Here the scenario is
If RS1 has not tranitioned the node from OFFLINE to OPENING if
RS2 gets the call from Master after forcefully chaning to OFFLINE the
Rs2 will take the control of the node.
At that time if RS1 starts transmitting the node then we should not allow
it.
===============================================================================


This addresses bug HBASE-4015.
    https://issues.apache.org/jira/browse/HBASE-4015


Diffs (updated)
-----

  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java 1161985 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 1161985 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 1161985 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1161985 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java PRE-CREATION 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1161985 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java 1161985 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java 1161985 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java 1161985 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java 1161985 

Diff: https://reviews.apache.org/r/1668/diff


Testing
-------

Yes.  But could not add new test case.
TestMasterFailOver is passing with the current changes also.


Thanks,

ramkrishna



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "stack (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

stack updated HBASE-4015:
-------------------------

      Resolution: Fixed
    Hadoop Flags: [Reviewed]
          Status: Resolved  (was: Patch Available)

Applied to TRUNK.  We should consider doing a version of this on branch.  J-D points out it changes the HRegionInterface.  Maybe if we put the change on the end in branch we'll be able to do rolling restarts up to 0.90.5.  I'll open new issue to look into this.

Thanks for persevering with the patch Ram.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Jean-Daniel Cryans (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13099637#comment-13099637 ] 

Jean-Daniel Cryans commented on HBASE-4015:
-------------------------------------------

@Ted, thanks I didn't see it amid the rest.

@Ram, did you insert any data in those regions before killing the RSs? Replaying the edits usually a good chunk of time for the region to be reopened. You could also try doing a worst case cold startup by killing -9 all HBase components at the same time (more or less) and then restarting them all (also after data was added). Finally you could try setting a super low timeout setting, like 5 seconds, to trigger RIT timeouts by the hundreds.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13088368#comment-13088368 ] 

Ted Yu commented on HBASE-4015:
-------------------------------

Patch 2 is much bigger.
Personally I think adding a new state is simpler to code / maintain than changing openRegion() signature in HRegionInterface - considering the other region related methods don't require version parameter.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13079990#comment-13079990 ] 

Ted Yu commented on HBASE-4015:
-------------------------------

Nice writeup, Ramkrishna.
bq. If the RE_ALLOCATE state gets timeout we will again call RE_ALLOCATE with new RS address.
What if timeout happens for all available region servers ?
We should prevent region from bouncing endlessly.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13098991#comment-13098991 ] 

jiraposter@reviews.apache.org commented on HBASE-4015:
------------------------------------------------------



bq.  On 2011-09-07 13:55:00, Ted Yu wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1260
bq.  > <https://reviews.apache.org/r/1668/diff/3/?file=36181#file36181line1260>
bq.  >
bq.  >     I think preempt alone would suffice.
bq.  >     We don't need to place hijack and preempt together.

I can do this on commit .. no need to change Ram.


- Michael


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/1668/#review1786
-----------------------------------------------------------


On 2011-08-29 12:11:23, ramkrishna vasudevan wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/1668/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2011-08-29 12:11:23)
bq.  
bq.  
bq.  Review request for Ted Yu, Michael Stack, Jean-Daniel Cryans, and Jonathan Gray.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  HBASE-4015 - updated patch.
bq.  
bq.  
bq.          invokeTimeOutManager(regionState.getRegion(),
bq.              TimeOutOperationType.ASSIGN);
bq.  
bq.  Instead of storing the state when timeout was deducted we process it
bq.  using future task.
bq.  ================================================================
bq.  
bq.        case RS_ZK_REGION_OPENING:
bq.          // TODO: Could check if it was on deadServers.  If it was, then we could
bq.          // do what happens in TimeoutMonitor when it sees this condition.
bq.  
bq.          // Just insert region into RIT
bq.          // If this never updates the timeout will trigger new assignment
bq.          if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
bq.            regionsInTransition.put(encodedRegionName, new RegionState(
bq.                regionInfo, RegionState.State.OPENING, data.getStamp(), data
bq.                    .getOrigin()));
bq.            processOpeningState(regionInfo);
bq.            break;
bq.          }
bq.          regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
bq.              RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
bq.          break;
bq.  
bq.  This change is for HBASE-4203. META and ROOT table need not wait till timeout
bq.  ======================================================================
bq.  
bq.  In forceRegionStateToOffline()
bq.  
bq.      } else {
bq.        // If invoked from timeout monitor donot force it to OFFLINE. Based on the
bq.        // state we will decide if to change in-memory state to OFFLINE or not.  It will
bq.        // be done before setting the znode to OFFLINE state.
bq.        if (!timeOutMonitorReAllocate) {
bq.          LOG.debug("Forcing OFFLINE; was=" + state);
bq.          state.update(RegionState.State.OFFLINE);
bq.        }
bq.  If the timeout monitor tries to reallcoate the node then dont make
bq.  the inmemory state to OFFLINE.
bq.  But the noraml assign flow doesnot expect the inmemory state to OFFLINE.
bq.  Hence the above change.  This is continued with the check in 
bq.  assign()
bq.  int setOfflineInZooKeeper(final RegionState state,
bq.        boolean timeOutMonitorReAllocate) {
bq.      // If invoked from timeoutmonitor the current state in memory need not be
bq.      // OFFLINE.  
bq.      if (!timeOutMonitorReAllocate && !state.isClosed() && !state.isOffline()) {
bq.            this.master.abort("Unexpected state trying to OFFLINE; " + state,
bq.            new IllegalStateException());
bq.        return -1;
bq.      }
bq.  ======================================================================
bq.  
bq.      boolean allowCreation = false;
bq.      // If the isReAllocate is true and the current state is PENDING_OPEN
bq.      // or OPENING then update the inmemory state to PENDING_OPEN. This is
bq.      // important because
bq.      // if timeoutmonitor deducts that a region was in OPENING state for a long
bq.      // time but by the
bq.      // time timeout monitor tranits the node to OFFLINE the RS would have opened
bq.      // the node and the
bq.      // state in znode will be RS_ZK_REGION_OPENED. Inorder to invoke the
bq.      // OpenedRegionHandler
bq.      // we expect the inmemeory state to be PENDING_OPEN or OPENING.
bq.      // For all other cases we can change the inmemory state to OFFLINE.
bq.      if (timeOutMonitorReAllocate
bq.          && (state.getState().equals(RegionState.State.PENDING_OPEN) || state
bq.              .getState().equals(RegionState.State.OPENING))) {
bq.        state.update(RegionState.State.PENDING_OPEN);
bq.        allowCreation = false;
bq.      } else {
bq.        state.update(RegionState.State.OFFLINE);
bq.        allowCreation = true;
bq.      }
bq.  This change is quite tricky.  
bq.  In normal assign flow the unassigned node for the region will not be present
bq.  Hence we need to allow the creation of the node newly.
bq.  But in timeout monitor case we will have the node present in some state hence 
bq.  we decide whether to create node newly or not inside ZKAssign.createOrForceNodeOffline
bq.  
bq.  The above code also updates the inmemory state of OFFLINE or PENDING_OPEN
bq.  which was not update in the previous forceRegionStateToOffline() call.
bq.  ==============================================================================
bq.  In ZKAssign.java()
bq.      if (version == -1) {
bq.        // If timeoutmonitor deducts a node to be in OPENING state but before it
bq.        // could
bq.        // transit to OFFLINE state if RS had opened the region then the Master
bq.        // deletes the
bq.        // assigned region znode. In that case the znode will not exist. So we
bq.        // should not
bq.        // create the znode again which will lead to double assignment.
bq.        if (timeOutMonitorReAllocate && !allowCreation) {
bq.          return -1;
bq.        }
bq.  this part prevents double assignment
bq.  If timeoutmonitor tries to force to OFFLINE state an existing region which was in RIT
bq.  but before it could do that if the node was opened then openedregionhandler will delete
bq.  the node hence we should not create the node again.
bq.  
bq.  =======================================================================================
bq.  In ZkAssign.java()
bq.      } else {
bq.        RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
bq.            .getEncodedName(), stat);
bq.        // Do not move the node to OFFLINE if znode is in any of the following
bq.        // state.
bq.        // Because these are already executed states.
bq.        if (timeOutMonitorReAllocate && null != curDataInZNode) {
bq.          EventType eventType = curDataInZNode.getEventType();
bq.          if (eventType.equals(EventType.RS_ZK_REGION_CLOSING)
bq.              || eventType.equals(EventType.RS_ZK_REGION_CLOSED)
bq.              || eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
bq.            return -1;
bq.          }
bq.        }
bq.  This check prevents from moving the node to OFFLINE state if just before
bq.  the node is tried to force to OFFLINE the RS would have changed the state 
bq.  to either CLOSING or CLOSED or OPENED.  now again moving to OFFLINE
bq.  will lead to douoble assignment and will an additional operaition.
bq.  
bq.  ====================================================================================
bq.  In ZKassign.java()
bq.        boolean setData = false;
bq.        try {
bq.          setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
bq.          // Setdata throws KeeperException which aborts the Master. So we are
bq.          // catching it here.
bq.          // If just before setting the znode to OFFLINE if the RS has made any
bq.          // change to the
bq.          // znode state then we need to return -1.
bq.        } catch (KeeperException kpe) {
bq.          LOG.info("Version mismatch while setting the node to OFFLINE state.");
bq.          return -1;
bq.        }
bq.  This change is actually to avoid the master from abortng. If the forceful OFFLINE is in
bq.  progrss but just before setting the RS has changed the state.
bq.  Then the setData will fail leading to master abort.
bq.  Hence we are catching the exception.
bq.  ====================================================================================
bq.  
bq.  In assignmentManager.java
bq.  +      if (setOfflineInZK && versionOfOfflineNode == -1)
bq.  +        return;
bq.  This is nothing but the refactoring done in the existing code.
bq.  -     if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
bq.  So if setting the version is unsuccessful return.
bq.  =====================================================================================
bq.  In ZKassign.java()
bq.  // the below check ensure that double assignment doesnot happen.
bq.      // When the node is created for the first time then the expected version
bq.      // that is
bq.      // passed will be -1 and the version in znode will be 0.
bq.      // In all other cases the version in znode will be > 0.
bq.      else if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
bq.          && endState.equals(EventType.RS_ZK_REGION_OPENING)
bq.          && expectedVersion == -1 && stat.getVersion() != 0) {
bq.        LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
bq.            + encoded + " from " + beginState + " to " + endState + " failed, "
bq.            + "the node existed but was version " + stat.getVersion()
bq.            + " not the expected version " + expectedVersion));
bq.        return -1;
bq.      }
bq.  As the comment explains when the node is created for first time the expectedversion will
bq.  be -1 but the actual version will be 0.  Here the scenario is
bq.  If RS1 has not tranitioned the node from OFFLINE to OPENING if
bq.  RS2 gets the call from Master after forcefully chaning to OFFLINE the
bq.  Rs2 will take the control of the node.
bq.  At that time if RS1 starts transmitting the node then we should not allow
bq.  it.
bq.  ===============================================================================
bq.  
bq.  
bq.  This addresses bug HBASE-4015.
bq.      https://issues.apache.org/jira/browse/HBASE-4015
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java PRE-CREATION 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java 1161985 
bq.  
bq.  Diff: https://reviews.apache.org/r/1668/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  Yes.  But could not add new test case.
bq.  TestMasterFailOver is passing with the current changes also.
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  ramkrishna
bq.  
bq.



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13083965#comment-13083965 ] 

stack commented on HBASE-4015:
------------------------------

bq. Timeout monitor DOESNOT preempt an znode to OFFLINE if in PENDING_OPEN state.

Ok.

I think I understand now.  The addition of new state breaks the move to OPENING because the check for a previous OFFLINE state will fail... so the RS will no proceed with the open.

But in fig (iii) in your doc. you check previous state is REALLOCATE? How is this case different from the fig (i) where you check for OFFLINE?  Won't your code have to check for both REALLOCATE and OFFLINE and the presence of either mean its ok to procede to OPENING (and then aren't REALLOCATE and OFFLINE the 'same' state because the presence of either will mean proceed to OPENING?).

I suppose the presence of the RS name will help.  If its the 'same' name, then we can proceed to OPENING and so what if OFFLINE was hijacked and became a REALLOCATE.  If they are not the same, then we'd abort the open.

So, why not just add machine name to OFFLINE?  Then we don't need REALLOCATE state? (Ideally it would be best if master told the regionserver the version of the znode to expect when it goes to move the znode to OPENING but that looks hard to pass from the master over to the RS EventHandlers).

So, figuring how to do deal with timeout of regions in PENDING_OPEN is one aspect of this issue, right?  The verification of state over in timeout monitor before acting is another aspect?

You are working on TRUNK Ram? (I believe it acts a little differently from 0.90 because of recent work done in here).

Good stuff Ram.  Thanks for digging into this.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13080006#comment-13080006 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

@Ted,
Thanks for your review and comments.
So the point here is if the Region is not getting assigned(or allocated) at all even after changing the state to RE_ALLOCATE many times do we throw some error?
So the number of times RE_ALLOCATE is done should be configurable after which we need to throw error.  Correct me if am wrong Ted.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13082370#comment-13082370 ] 

Ted Yu commented on HBASE-4015:
-------------------------------

{code}
+      CatalogTracker catalogTracker, final ExecutorService service, final java.util.concurrent.ExecutorService threadPoolExecutorService)
{code}
Wrap line please.
Why do we use two ExecutorService classes ?

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

ramkrishna.s.vasudevan updated HBASE-4015:
------------------------------------------

    Attachment: HBASE-4015_2_trunk.patch

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13097008#comment-13097008 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

I ran the hbck tool.  The result shows 0 inconsistencies.  Infact over the week end gave a long run.  Killing different RS.
 
Ran the HBCK tool and the results were positive.  Also restarted the entire cluster and again ran HBCK even then the HBCK showed no
inconsistencies. 
Thanks Ted and Stack for your review.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13091796#comment-13091796 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

The patch for HBASE-4203 is also present in the same one.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13090025#comment-13090025 ] 

Ted Yu commented on HBASE-4015:
-------------------------------

In TRUNK, HRegionInfo.isMetaRegion() only checks for .META.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13092148#comment-13092148 ] 

jiraposter@reviews.apache.org commented on HBASE-4015:
------------------------------------------------------


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/1668/#review1666
-----------------------------------------------------------


Looks great Ram.  WOuld be hard to write a test for this I think.  What do you think?


http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
<https://reviews.apache.org/r/1668/#comment3719>

    White space.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
<https://reviews.apache.org/r/1668/#comment3720>

    Is it the RS that is creating the OFFLINE node or is it the master that does this and RS moves the region from OFFLINE to OPENING?  No need to say 'passed by the master'



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
<https://reviews.apache.org/r/1668/#comment3721>

    Is this correct?  The open is async w/ the states of opening done up in zk w/ master noticing the last -- right?  The return is whether or not the request to open was accepted?



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
<https://reviews.apache.org/r/1668/#comment3722>

    ws



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java
<https://reviews.apache.org/r/1668/#comment3723>

    ws



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3724>

    ws



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3725>

    ws



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3726>

    So, whats this special-handling of catalog tables about?  Want to add comment why it is so?



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3727>

    Todd would say don't bother adding javadoc if it says nothing other than repeat param names.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3728>

    This is a better name for this param though why does this method need to know who is calling it?  Perhaps there is a better name?  timedout?  What you think Ram?
    
    Do you mean 'reassign' when you say 'reallocate'
    
    So maybe use 'reassign' instead of 'timedout'?



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3729>

    ws



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3730>

    ws



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3731>

    When do we do this?  Where do we do the change of in-memory state?



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3732>

    ws



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3733>

    yeah, why should this method care where its called from so would suggest removing timeoutMonitor from main and name param for condition under which this method is being called.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3738>

    Good.  So if an open comes in on a table that is being disabled, we'll skip... thats right.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3734>

    ws



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3736>

    ws



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3737>

    Excellent



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3739>

    ws



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3740>

    ws



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3745>

    maybe 'allowZNodeCreation' instead?



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3742>

    When you say 'deducts' do you mean to use the current tense?  If so, that'd be 'deduces'.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3741>

    Can you fix up this comment so it aligns a bit better?
    
    There is some misspelling in this paragraph that you might want to fix?



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3744>

    Can you format this a little better?  Have first line end with && and then second line end with ||.  The state.getState....should be on a single line ... make it easier to read rather than split over two lines.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3746>

    So, what you think will happen next?  We'll fail the assign?  Region will be stuck?



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3748>

    Its good breaking down this method....



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3749>

    FYI, I prefer the former formatting where the arg was all on one line rather than split over two.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3750>

    good



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3751>

    ws



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3752>

    Did you change these Excecutors to verify that the state that caused their being queued still holds before they actually run?  That would be important I'd think if you have not done it that is.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
<https://reviews.apache.org/r/1668/#comment3753>

    ws



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java
<https://reviews.apache.org/r/1668/#comment3754>

    why extra lines down here?



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
<https://reviews.apache.org/r/1668/#comment3755>

    ws



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java
<https://reviews.apache.org/r/1668/#comment3756>

    ws



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
<https://reviews.apache.org/r/1668/#comment3757>

    ws



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
<https://reviews.apache.org/r/1668/#comment3758>

    ws


- Michael


On 2011-08-26 14:23:24, ramkrishna vasudevan wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/1668/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2011-08-26 14:23:24)
bq.  
bq.  
bq.  Review request for Ted Yu, Michael Stack, Jean-Daniel Cryans, and Jonathan Gray.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  HBASE-4015 - updated patch.
bq.  
bq.  
bq.          invokeTimeOutManager(regionState.getRegion(),
bq.              TimeOutOperationType.ASSIGN);
bq.  
bq.  Instead of storing the state when timeout was deducted we process it
bq.  using future task.
bq.  ================================================================
bq.  
bq.        case RS_ZK_REGION_OPENING:
bq.          // TODO: Could check if it was on deadServers.  If it was, then we could
bq.          // do what happens in TimeoutMonitor when it sees this condition.
bq.  
bq.          // Just insert region into RIT
bq.          // If this never updates the timeout will trigger new assignment
bq.          if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
bq.            regionsInTransition.put(encodedRegionName, new RegionState(
bq.                regionInfo, RegionState.State.OPENING, data.getStamp(), data
bq.                    .getOrigin()));
bq.            processOpeningState(regionInfo);
bq.            break;
bq.          }
bq.          regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
bq.              RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
bq.          break;
bq.  
bq.  This change is for HBASE-4203. META and ROOT table need not wait till timeout
bq.  ======================================================================
bq.  
bq.  In forceRegionStateToOffline()
bq.  
bq.      } else {
bq.        // If invoked from timeout monitor donot force it to OFFLINE. Based on the
bq.        // state we will decide if to change in-memory state to OFFLINE or not.  It will
bq.        // be done before setting the znode to OFFLINE state.
bq.        if (!timeOutMonitorReAllocate) {
bq.          LOG.debug("Forcing OFFLINE; was=" + state);
bq.          state.update(RegionState.State.OFFLINE);
bq.        }
bq.  If the timeout monitor tries to reallcoate the node then dont make
bq.  the inmemory state to OFFLINE.
bq.  But the noraml assign flow doesnot expect the inmemory state to OFFLINE.
bq.  Hence the above change.  This is continued with the check in 
bq.  assign()
bq.  int setOfflineInZooKeeper(final RegionState state,
bq.        boolean timeOutMonitorReAllocate) {
bq.      // If invoked from timeoutmonitor the current state in memory need not be
bq.      // OFFLINE.  
bq.      if (!timeOutMonitorReAllocate && !state.isClosed() && !state.isOffline()) {
bq.            this.master.abort("Unexpected state trying to OFFLINE; " + state,
bq.            new IllegalStateException());
bq.        return -1;
bq.      }
bq.  ======================================================================
bq.  
bq.      boolean allowCreation = false;
bq.      // If the isReAllocate is true and the current state is PENDING_OPEN
bq.      // or OPENING then update the inmemory state to PENDING_OPEN. This is
bq.      // important because
bq.      // if timeoutmonitor deducts that a region was in OPENING state for a long
bq.      // time but by the
bq.      // time timeout monitor tranits the node to OFFLINE the RS would have opened
bq.      // the node and the
bq.      // state in znode will be RS_ZK_REGION_OPENED. Inorder to invoke the
bq.      // OpenedRegionHandler
bq.      // we expect the inmemeory state to be PENDING_OPEN or OPENING.
bq.      // For all other cases we can change the inmemory state to OFFLINE.
bq.      if (timeOutMonitorReAllocate
bq.          && (state.getState().equals(RegionState.State.PENDING_OPEN) || state
bq.              .getState().equals(RegionState.State.OPENING))) {
bq.        state.update(RegionState.State.PENDING_OPEN);
bq.        allowCreation = false;
bq.      } else {
bq.        state.update(RegionState.State.OFFLINE);
bq.        allowCreation = true;
bq.      }
bq.  This change is quite tricky.  
bq.  In normal assign flow the unassigned node for the region will not be present
bq.  Hence we need to allow the creation of the node newly.
bq.  But in timeout monitor case we will have the node present in some state hence 
bq.  we decide whether to create node newly or not inside ZKAssign.createOrForceNodeOffline
bq.  
bq.  The above code also updates the inmemory state of OFFLINE or PENDING_OPEN
bq.  which was not update in the previous forceRegionStateToOffline() call.
bq.  ==============================================================================
bq.  In ZKAssign.java()
bq.      if (version == -1) {
bq.        // If timeoutmonitor deducts a node to be in OPENING state but before it
bq.        // could
bq.        // transit to OFFLINE state if RS had opened the region then the Master
bq.        // deletes the
bq.        // assigned region znode. In that case the znode will not exist. So we
bq.        // should not
bq.        // create the znode again which will lead to double assignment.
bq.        if (timeOutMonitorReAllocate && !allowCreation) {
bq.          return -1;
bq.        }
bq.  this part prevents double assignment
bq.  If timeoutmonitor tries to force to OFFLINE state an existing region which was in RIT
bq.  but before it could do that if the node was opened then openedregionhandler will delete
bq.  the node hence we should not create the node again.
bq.  
bq.  =======================================================================================
bq.  In ZkAssign.java()
bq.      } else {
bq.        RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
bq.            .getEncodedName(), stat);
bq.        // Do not move the node to OFFLINE if znode is in any of the following
bq.        // state.
bq.        // Because these are already executed states.
bq.        if (timeOutMonitorReAllocate && null != curDataInZNode) {
bq.          EventType eventType = curDataInZNode.getEventType();
bq.          if (eventType.equals(EventType.RS_ZK_REGION_CLOSING)
bq.              || eventType.equals(EventType.RS_ZK_REGION_CLOSED)
bq.              || eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
bq.            return -1;
bq.          }
bq.        }
bq.  This check prevents from moving the node to OFFLINE state if just before
bq.  the node is tried to force to OFFLINE the RS would have changed the state 
bq.  to either CLOSING or CLOSED or OPENED.  now again moving to OFFLINE
bq.  will lead to douoble assignment and will an additional operaition.
bq.  
bq.  ====================================================================================
bq.  In ZKassign.java()
bq.        boolean setData = false;
bq.        try {
bq.          setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
bq.          // Setdata throws KeeperException which aborts the Master. So we are
bq.          // catching it here.
bq.          // If just before setting the znode to OFFLINE if the RS has made any
bq.          // change to the
bq.          // znode state then we need to return -1.
bq.        } catch (KeeperException kpe) {
bq.          LOG.info("Version mismatch while setting the node to OFFLINE state.");
bq.          return -1;
bq.        }
bq.  This change is actually to avoid the master from abortng. If the forceful OFFLINE is in
bq.  progrss but just before setting the RS has changed the state.
bq.  Then the setData will fail leading to master abort.
bq.  Hence we are catching the exception.
bq.  ====================================================================================
bq.  
bq.  In assignmentManager.java
bq.  +      if (setOfflineInZK && versionOfOfflineNode == -1)
bq.  +        return;
bq.  This is nothing but the refactoring done in the existing code.
bq.  -     if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
bq.  So if setting the version is unsuccessful return.
bq.  =====================================================================================
bq.  In ZKassign.java()
bq.  // the below check ensure that double assignment doesnot happen.
bq.      // When the node is created for the first time then the expected version
bq.      // that is
bq.      // passed will be -1 and the version in znode will be 0.
bq.      // In all other cases the version in znode will be > 0.
bq.      else if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
bq.          && endState.equals(EventType.RS_ZK_REGION_OPENING)
bq.          && expectedVersion == -1 && stat.getVersion() != 0) {
bq.        LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
bq.            + encoded + " from " + beginState + " to " + endState + " failed, "
bq.            + "the node existed but was version " + stat.getVersion()
bq.            + " not the expected version " + expectedVersion));
bq.        return -1;
bq.      }
bq.  As the comment explains when the node is created for first time the expectedversion will
bq.  be -1 but the actual version will be 0.  Here the scenario is
bq.  If RS1 has not tranitioned the node from OFFLINE to OPENING if
bq.  RS2 gets the call from Master after forcefully chaning to OFFLINE the
bq.  Rs2 will take the control of the node.
bq.  At that time if RS1 starts transmitting the node then we should not allow
bq.  it.
bq.  ===============================================================================
bq.  
bq.  
bq.  This addresses bug HBASE-4015.
bq.      https://issues.apache.org/jira/browse/HBASE-4015
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java PRE-CREATION 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java 1161985 
bq.  
bq.  Diff: https://reviews.apache.org/r/1668/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  Yes.  But could not add new test case.
bq.  TestMasterFailOver is passing with the current changes also.
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  ramkrishna
bq.  
bq.



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13080355#comment-13080355 ] 

Ted Yu commented on HBASE-4015:
-------------------------------

bq. Also if the RS1 which was not able to process OFFLINE to OPENING finds that its state has changed to RE_ALLOCATE ...
I think there is a short window for race condition here:
1. master tries to change state to RE_ALLOCATE
2. RS1 recovers and tries to change state to OPENING
1 and 2 may happen at the same time.


> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13099048#comment-13099048 ] 

jiraposter@reviews.apache.org commented on HBASE-4015:
------------------------------------------------------


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/1668/
-----------------------------------------------------------

(Updated 2011-09-07 15:52:26.864199)


Review request for Ted Yu, Michael Stack, Jean-Daniel Cryans, and Jonathan Gray.


Changes
-------

Updated the patch as per the review comments.
Changed hijackAndPreempt to hijack.


Summary
-------

HBASE-4015 - updated patch.


        invokeTimeOutManager(regionState.getRegion(),
            TimeOutOperationType.ASSIGN);

Instead of storing the state when timeout was deducted we process it
using future task.
================================================================

      case RS_ZK_REGION_OPENING:
        // TODO: Could check if it was on deadServers.  If it was, then we could
        // do what happens in TimeoutMonitor when it sees this condition.

        // Just insert region into RIT
        // If this never updates the timeout will trigger new assignment
        if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
          regionsInTransition.put(encodedRegionName, new RegionState(
              regionInfo, RegionState.State.OPENING, data.getStamp(), data
                  .getOrigin()));
          processOpeningState(regionInfo);
          break;
        }
        regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
            RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
        break;

This change is for HBASE-4203. META and ROOT table need not wait till timeout
======================================================================

In forceRegionStateToOffline()

    } else {
      // If invoked from timeout monitor donot force it to OFFLINE. Based on the
      // state we will decide if to change in-memory state to OFFLINE or not.  It will
      // be done before setting the znode to OFFLINE state.
      if (!timeOutMonitorReAllocate) {
        LOG.debug("Forcing OFFLINE; was=" + state);
        state.update(RegionState.State.OFFLINE);
      }
If the timeout monitor tries to reallcoate the node then dont make
the inmemory state to OFFLINE.
But the noraml assign flow doesnot expect the inmemory state to OFFLINE.
Hence the above change.  This is continued with the check in 
assign()
int setOfflineInZooKeeper(final RegionState state,
      boolean timeOutMonitorReAllocate) {
    // If invoked from timeoutmonitor the current state in memory need not be
    // OFFLINE.  
    if (!timeOutMonitorReAllocate && !state.isClosed() && !state.isOffline()) {
          this.master.abort("Unexpected state trying to OFFLINE; " + state,
          new IllegalStateException());
      return -1;
    }
======================================================================

    boolean allowCreation = false;
    // If the isReAllocate is true and the current state is PENDING_OPEN
    // or OPENING then update the inmemory state to PENDING_OPEN. This is
    // important because
    // if timeoutmonitor deducts that a region was in OPENING state for a long
    // time but by the
    // time timeout monitor tranits the node to OFFLINE the RS would have opened
    // the node and the
    // state in znode will be RS_ZK_REGION_OPENED. Inorder to invoke the
    // OpenedRegionHandler
    // we expect the inmemeory state to be PENDING_OPEN or OPENING.
    // For all other cases we can change the inmemory state to OFFLINE.
    if (timeOutMonitorReAllocate
        && (state.getState().equals(RegionState.State.PENDING_OPEN) || state
            .getState().equals(RegionState.State.OPENING))) {
      state.update(RegionState.State.PENDING_OPEN);
      allowCreation = false;
    } else {
      state.update(RegionState.State.OFFLINE);
      allowCreation = true;
    }
This change is quite tricky.  
In normal assign flow the unassigned node for the region will not be present
Hence we need to allow the creation of the node newly.
But in timeout monitor case we will have the node present in some state hence 
we decide whether to create node newly or not inside ZKAssign.createOrForceNodeOffline

The above code also updates the inmemory state of OFFLINE or PENDING_OPEN
which was not update in the previous forceRegionStateToOffline() call.
==============================================================================
In ZKAssign.java()
    if (version == -1) {
      // If timeoutmonitor deducts a node to be in OPENING state but before it
      // could
      // transit to OFFLINE state if RS had opened the region then the Master
      // deletes the
      // assigned region znode. In that case the znode will not exist. So we
      // should not
      // create the znode again which will lead to double assignment.
      if (timeOutMonitorReAllocate && !allowCreation) {
        return -1;
      }
this part prevents double assignment
If timeoutmonitor tries to force to OFFLINE state an existing region which was in RIT
but before it could do that if the node was opened then openedregionhandler will delete
the node hence we should not create the node again.

=======================================================================================
In ZkAssign.java()
    } else {
      RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
          .getEncodedName(), stat);
      // Do not move the node to OFFLINE if znode is in any of the following
      // state.
      // Because these are already executed states.
      if (timeOutMonitorReAllocate && null != curDataInZNode) {
        EventType eventType = curDataInZNode.getEventType();
        if (eventType.equals(EventType.RS_ZK_REGION_CLOSING)
            || eventType.equals(EventType.RS_ZK_REGION_CLOSED)
            || eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
          return -1;
        }
      }
This check prevents from moving the node to OFFLINE state if just before
the node is tried to force to OFFLINE the RS would have changed the state 
to either CLOSING or CLOSED or OPENED.  now again moving to OFFLINE
will lead to douoble assignment and will an additional operaition.

====================================================================================
In ZKassign.java()
      boolean setData = false;
      try {
        setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
        // Setdata throws KeeperException which aborts the Master. So we are
        // catching it here.
        // If just before setting the znode to OFFLINE if the RS has made any
        // change to the
        // znode state then we need to return -1.
      } catch (KeeperException kpe) {
        LOG.info("Version mismatch while setting the node to OFFLINE state.");
        return -1;
      }
This change is actually to avoid the master from abortng. If the forceful OFFLINE is in
progrss but just before setting the RS has changed the state.
Then the setData will fail leading to master abort.
Hence we are catching the exception.
====================================================================================

In assignmentManager.java
+      if (setOfflineInZK && versionOfOfflineNode == -1)
+        return;
This is nothing but the refactoring done in the existing code.
-     if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
So if setting the version is unsuccessful return.
=====================================================================================
In ZKassign.java()
// the below check ensure that double assignment doesnot happen.
    // When the node is created for the first time then the expected version
    // that is
    // passed will be -1 and the version in znode will be 0.
    // In all other cases the version in znode will be > 0.
    else if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
        && endState.equals(EventType.RS_ZK_REGION_OPENING)
        && expectedVersion == -1 && stat.getVersion() != 0) {
      LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
          + encoded + " from " + beginState + " to " + endState + " failed, "
          + "the node existed but was version " + stat.getVersion()
          + " not the expected version " + expectedVersion));
      return -1;
    }
As the comment explains when the node is created for first time the expectedversion will
be -1 but the actual version will be 0.  Here the scenario is
If RS1 has not tranitioned the node from OFFLINE to OPENING if
RS2 gets the call from Master after forcefully chaning to OFFLINE the
Rs2 will take the control of the node.
At that time if RS1 starts transmitting the node then we should not allow
it.
===============================================================================


This addresses bug HBASE-4015.
    https://issues.apache.org/jira/browse/HBASE-4015


Diffs (updated)
-----

  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 1166224 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignCallable.java PRE-CREATION 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java 1166224 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 1166224 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1166224 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/UnAssignCallable.java PRE-CREATION 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1166224 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java 1166224 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java 1166224 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java 1166224 
  http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java 1166224 

Diff: https://reviews.apache.org/r/1668/diff


Testing
-------

Yes.  But could not add new test case.
TestMasterFailOver is passing with the current changes also.


Thanks,

ramkrishna



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13092171#comment-13092171 ] 

jiraposter@reviews.apache.org commented on HBASE-4015:
------------------------------------------------------


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/1668/#review1668
-----------------------------------------------------------



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment3770>

    The two strings can be concatenated:
     region close", 


- Ted


On 2011-08-26 14:23:24, ramkrishna vasudevan wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/1668/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2011-08-26 14:23:24)
bq.  
bq.  
bq.  Review request for Ted Yu, Michael Stack, Jean-Daniel Cryans, and Jonathan Gray.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  HBASE-4015 - updated patch.
bq.  
bq.  
bq.          invokeTimeOutManager(regionState.getRegion(),
bq.              TimeOutOperationType.ASSIGN);
bq.  
bq.  Instead of storing the state when timeout was deducted we process it
bq.  using future task.
bq.  ================================================================
bq.  
bq.        case RS_ZK_REGION_OPENING:
bq.          // TODO: Could check if it was on deadServers.  If it was, then we could
bq.          // do what happens in TimeoutMonitor when it sees this condition.
bq.  
bq.          // Just insert region into RIT
bq.          // If this never updates the timeout will trigger new assignment
bq.          if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
bq.            regionsInTransition.put(encodedRegionName, new RegionState(
bq.                regionInfo, RegionState.State.OPENING, data.getStamp(), data
bq.                    .getOrigin()));
bq.            processOpeningState(regionInfo);
bq.            break;
bq.          }
bq.          regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
bq.              RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
bq.          break;
bq.  
bq.  This change is for HBASE-4203. META and ROOT table need not wait till timeout
bq.  ======================================================================
bq.  
bq.  In forceRegionStateToOffline()
bq.  
bq.      } else {
bq.        // If invoked from timeout monitor donot force it to OFFLINE. Based on the
bq.        // state we will decide if to change in-memory state to OFFLINE or not.  It will
bq.        // be done before setting the znode to OFFLINE state.
bq.        if (!timeOutMonitorReAllocate) {
bq.          LOG.debug("Forcing OFFLINE; was=" + state);
bq.          state.update(RegionState.State.OFFLINE);
bq.        }
bq.  If the timeout monitor tries to reallcoate the node then dont make
bq.  the inmemory state to OFFLINE.
bq.  But the noraml assign flow doesnot expect the inmemory state to OFFLINE.
bq.  Hence the above change.  This is continued with the check in 
bq.  assign()
bq.  int setOfflineInZooKeeper(final RegionState state,
bq.        boolean timeOutMonitorReAllocate) {
bq.      // If invoked from timeoutmonitor the current state in memory need not be
bq.      // OFFLINE.  
bq.      if (!timeOutMonitorReAllocate && !state.isClosed() && !state.isOffline()) {
bq.            this.master.abort("Unexpected state trying to OFFLINE; " + state,
bq.            new IllegalStateException());
bq.        return -1;
bq.      }
bq.  ======================================================================
bq.  
bq.      boolean allowCreation = false;
bq.      // If the isReAllocate is true and the current state is PENDING_OPEN
bq.      // or OPENING then update the inmemory state to PENDING_OPEN. This is
bq.      // important because
bq.      // if timeoutmonitor deducts that a region was in OPENING state for a long
bq.      // time but by the
bq.      // time timeout monitor tranits the node to OFFLINE the RS would have opened
bq.      // the node and the
bq.      // state in znode will be RS_ZK_REGION_OPENED. Inorder to invoke the
bq.      // OpenedRegionHandler
bq.      // we expect the inmemeory state to be PENDING_OPEN or OPENING.
bq.      // For all other cases we can change the inmemory state to OFFLINE.
bq.      if (timeOutMonitorReAllocate
bq.          && (state.getState().equals(RegionState.State.PENDING_OPEN) || state
bq.              .getState().equals(RegionState.State.OPENING))) {
bq.        state.update(RegionState.State.PENDING_OPEN);
bq.        allowCreation = false;
bq.      } else {
bq.        state.update(RegionState.State.OFFLINE);
bq.        allowCreation = true;
bq.      }
bq.  This change is quite tricky.  
bq.  In normal assign flow the unassigned node for the region will not be present
bq.  Hence we need to allow the creation of the node newly.
bq.  But in timeout monitor case we will have the node present in some state hence 
bq.  we decide whether to create node newly or not inside ZKAssign.createOrForceNodeOffline
bq.  
bq.  The above code also updates the inmemory state of OFFLINE or PENDING_OPEN
bq.  which was not update in the previous forceRegionStateToOffline() call.
bq.  ==============================================================================
bq.  In ZKAssign.java()
bq.      if (version == -1) {
bq.        // If timeoutmonitor deducts a node to be in OPENING state but before it
bq.        // could
bq.        // transit to OFFLINE state if RS had opened the region then the Master
bq.        // deletes the
bq.        // assigned region znode. In that case the znode will not exist. So we
bq.        // should not
bq.        // create the znode again which will lead to double assignment.
bq.        if (timeOutMonitorReAllocate && !allowCreation) {
bq.          return -1;
bq.        }
bq.  this part prevents double assignment
bq.  If timeoutmonitor tries to force to OFFLINE state an existing region which was in RIT
bq.  but before it could do that if the node was opened then openedregionhandler will delete
bq.  the node hence we should not create the node again.
bq.  
bq.  =======================================================================================
bq.  In ZkAssign.java()
bq.      } else {
bq.        RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
bq.            .getEncodedName(), stat);
bq.        // Do not move the node to OFFLINE if znode is in any of the following
bq.        // state.
bq.        // Because these are already executed states.
bq.        if (timeOutMonitorReAllocate && null != curDataInZNode) {
bq.          EventType eventType = curDataInZNode.getEventType();
bq.          if (eventType.equals(EventType.RS_ZK_REGION_CLOSING)
bq.              || eventType.equals(EventType.RS_ZK_REGION_CLOSED)
bq.              || eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
bq.            return -1;
bq.          }
bq.        }
bq.  This check prevents from moving the node to OFFLINE state if just before
bq.  the node is tried to force to OFFLINE the RS would have changed the state 
bq.  to either CLOSING or CLOSED or OPENED.  now again moving to OFFLINE
bq.  will lead to douoble assignment and will an additional operaition.
bq.  
bq.  ====================================================================================
bq.  In ZKassign.java()
bq.        boolean setData = false;
bq.        try {
bq.          setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
bq.          // Setdata throws KeeperException which aborts the Master. So we are
bq.          // catching it here.
bq.          // If just before setting the znode to OFFLINE if the RS has made any
bq.          // change to the
bq.          // znode state then we need to return -1.
bq.        } catch (KeeperException kpe) {
bq.          LOG.info("Version mismatch while setting the node to OFFLINE state.");
bq.          return -1;
bq.        }
bq.  This change is actually to avoid the master from abortng. If the forceful OFFLINE is in
bq.  progrss but just before setting the RS has changed the state.
bq.  Then the setData will fail leading to master abort.
bq.  Hence we are catching the exception.
bq.  ====================================================================================
bq.  
bq.  In assignmentManager.java
bq.  +      if (setOfflineInZK && versionOfOfflineNode == -1)
bq.  +        return;
bq.  This is nothing but the refactoring done in the existing code.
bq.  -     if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
bq.  So if setting the version is unsuccessful return.
bq.  =====================================================================================
bq.  In ZKassign.java()
bq.  // the below check ensure that double assignment doesnot happen.
bq.      // When the node is created for the first time then the expected version
bq.      // that is
bq.      // passed will be -1 and the version in znode will be 0.
bq.      // In all other cases the version in znode will be > 0.
bq.      else if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
bq.          && endState.equals(EventType.RS_ZK_REGION_OPENING)
bq.          && expectedVersion == -1 && stat.getVersion() != 0) {
bq.        LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
bq.            + encoded + " from " + beginState + " to " + endState + " failed, "
bq.            + "the node existed but was version " + stat.getVersion()
bq.            + " not the expected version " + expectedVersion));
bq.        return -1;
bq.      }
bq.  As the comment explains when the node is created for first time the expectedversion will
bq.  be -1 but the actual version will be 0.  Here the scenario is
bq.  If RS1 has not tranitioned the node from OFFLINE to OPENING if
bq.  RS2 gets the call from Master after forcefully chaning to OFFLINE the
bq.  Rs2 will take the control of the node.
bq.  At that time if RS1 starts transmitting the node then we should not allow
bq.  it.
bq.  ===============================================================================
bq.  
bq.  
bq.  This addresses bug HBASE-4015.
bq.      https://issues.apache.org/jira/browse/HBASE-4015
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java PRE-CREATION 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java 1161985 
bq.  
bq.  Diff: https://reviews.apache.org/r/1668/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  Yes.  But could not add new test case.
bq.  TestMasterFailOver is passing with the current changes also.
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  ramkrishna
bq.  
bq.



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13082559#comment-13082559 ] 

stack commented on HBASE-4015:
------------------------------

bq. But we thought of introducing a new state so that there is a clear distinction whether reallocation has happened or not and also handling of the new state may be cleaner than changing the behaviour in the existing state.

I would not be worried about changing current states.  All of this stuff is transient in zk and it'll be in 0.92 requiring restart so change the states.

Adding new REALLOC state seems gratuitous (I don't see the diff from OFFLINE; OFFLINE+servername might help in some cases).  More states make it harder to chase down all transition scenarios.

Looking at the diagram again, I'm not sure it addresses the issue.

Do we even need the new state to address the core timeout monitor race issue?  The regionserver already is careful about checking states AND version number; i.e. if not expected state it will give up on opening or if not expected version it will close a region it has already opened.

The core prob. as per J-D above is that state transitions happen fine out on the regionserver but the master lags processing them; meantime the timeout monitor runs and presumes since its not seen the transition (that is likely in queue to process), it preempts znode setting it OFFLINE.





> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13084007#comment-13084007 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

bq. You are working on TRUNK Ram?
Yes Stack

bq. Won't your code have to check for both REALLOCATE and OFFLINE and the presence of either mean its ok to procede to OPENING (and then aren't REALLOCATE and OFFLINE the 'same' state because the presence of either will mean proceed to OPENING?).

Yes this is what my patch does.  But why we do the same operation for both state?
this is because previously if there is a change in state other than OFFLINE while moving to OPENING we were aborting, now this an additional state which says its ok to go to OPENING if you find me in RE_ALLOCATE and if the server name in me is same as your RS address. This avoids the problem of unnecessary region getting hijacked though the RS was doing his work correctly.

bq.So, why not just add machine name to OFFLINE? Then we don't need REALLOCATE state? 
This you have already told like currently there is no version that is passed from master to rs. Thats why a new state.  If this had been possible then OFFLINE with version passed by master would have been sufficient.

bq.So, figuring how to do deal with timeout of regions in PENDING_OPEN is one aspect of this issue, right? The verification of state over in timeout monitor before acting is another aspect?
Yes stack.. we have covered both these aspects and also the points told by JD.  Taking action on timeout immediately and a mechanism for both master and RS to know what happened as part of timeout and who ever wins the race succeeds.  

bq.(I believe it acts a little differently from 0.90 because of recent work done in here).

Reg timeout monitor the one major change is now the CLSOING state node is created by master itself and it was done by RS as in 0.90.  Apart from this i dint find any big difference till now. As part of HBASE-4083 we have introduced the return types from Open RegionHandler which takes care of scenarios where a race condition happens between the master changes to RE_ALLOCATE by the time the RS has moved to OPENED.



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13085533#comment-13085533 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

@Stack

I am ready to work together :).  Stack as in my previous comment I just got an highlevel points to be done if we stick with OFFLINE state itself. But I have not yet started my implementation.
So whats your idea Stack? Can i start digging as how many changes do we need to make if we go with OFFLINE state and what are the interface changes etc.  Or you have anything else in mind ?

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13092792#comment-13092792 ] 

jiraposter@reviews.apache.org commented on HBASE-4015:
------------------------------------------------------



bq.  On 2011-08-27 15:45:25, Ted Yu wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java, line 350
bq.  > <https://reviews.apache.org/r/1668/diff/2/?file=36049#file36049line350>
bq.  >
bq.  >     The three states should be placed on three lines with proper indentation.

updated


bq.  On 2011-08-27 15:45:25, Ted Yu wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1379
bq.  > <https://reviews.apache.org/r/1668/diff/2/?file=36050#file36050line1379>
bq.  >
bq.  >     versionOfTheOffline seems to be copied from a variable name. Please split it into 4 words: version of the offline

updated


- ramkrishna


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/1668/#review1672
-----------------------------------------------------------


On 2011-08-27 11:51:16, ramkrishna vasudevan wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/1668/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2011-08-27 11:51:16)
bq.  
bq.  
bq.  Review request for Ted Yu, Michael Stack, Jean-Daniel Cryans, and Jonathan Gray.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  HBASE-4015 - updated patch.
bq.  
bq.  
bq.          invokeTimeOutManager(regionState.getRegion(),
bq.              TimeOutOperationType.ASSIGN);
bq.  
bq.  Instead of storing the state when timeout was deducted we process it
bq.  using future task.
bq.  ================================================================
bq.  
bq.        case RS_ZK_REGION_OPENING:
bq.          // TODO: Could check if it was on deadServers.  If it was, then we could
bq.          // do what happens in TimeoutMonitor when it sees this condition.
bq.  
bq.          // Just insert region into RIT
bq.          // If this never updates the timeout will trigger new assignment
bq.          if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
bq.            regionsInTransition.put(encodedRegionName, new RegionState(
bq.                regionInfo, RegionState.State.OPENING, data.getStamp(), data
bq.                    .getOrigin()));
bq.            processOpeningState(regionInfo);
bq.            break;
bq.          }
bq.          regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
bq.              RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
bq.          break;
bq.  
bq.  This change is for HBASE-4203. META and ROOT table need not wait till timeout
bq.  ======================================================================
bq.  
bq.  In forceRegionStateToOffline()
bq.  
bq.      } else {
bq.        // If invoked from timeout monitor donot force it to OFFLINE. Based on the
bq.        // state we will decide if to change in-memory state to OFFLINE or not.  It will
bq.        // be done before setting the znode to OFFLINE state.
bq.        if (!timeOutMonitorReAllocate) {
bq.          LOG.debug("Forcing OFFLINE; was=" + state);
bq.          state.update(RegionState.State.OFFLINE);
bq.        }
bq.  If the timeout monitor tries to reallcoate the node then dont make
bq.  the inmemory state to OFFLINE.
bq.  But the noraml assign flow doesnot expect the inmemory state to OFFLINE.
bq.  Hence the above change.  This is continued with the check in 
bq.  assign()
bq.  int setOfflineInZooKeeper(final RegionState state,
bq.        boolean timeOutMonitorReAllocate) {
bq.      // If invoked from timeoutmonitor the current state in memory need not be
bq.      // OFFLINE.  
bq.      if (!timeOutMonitorReAllocate && !state.isClosed() && !state.isOffline()) {
bq.            this.master.abort("Unexpected state trying to OFFLINE; " + state,
bq.            new IllegalStateException());
bq.        return -1;
bq.      }
bq.  ======================================================================
bq.  
bq.      boolean allowCreation = false;
bq.      // If the isReAllocate is true and the current state is PENDING_OPEN
bq.      // or OPENING then update the inmemory state to PENDING_OPEN. This is
bq.      // important because
bq.      // if timeoutmonitor deducts that a region was in OPENING state for a long
bq.      // time but by the
bq.      // time timeout monitor tranits the node to OFFLINE the RS would have opened
bq.      // the node and the
bq.      // state in znode will be RS_ZK_REGION_OPENED. Inorder to invoke the
bq.      // OpenedRegionHandler
bq.      // we expect the inmemeory state to be PENDING_OPEN or OPENING.
bq.      // For all other cases we can change the inmemory state to OFFLINE.
bq.      if (timeOutMonitorReAllocate
bq.          && (state.getState().equals(RegionState.State.PENDING_OPEN) || state
bq.              .getState().equals(RegionState.State.OPENING))) {
bq.        state.update(RegionState.State.PENDING_OPEN);
bq.        allowCreation = false;
bq.      } else {
bq.        state.update(RegionState.State.OFFLINE);
bq.        allowCreation = true;
bq.      }
bq.  This change is quite tricky.  
bq.  In normal assign flow the unassigned node for the region will not be present
bq.  Hence we need to allow the creation of the node newly.
bq.  But in timeout monitor case we will have the node present in some state hence 
bq.  we decide whether to create node newly or not inside ZKAssign.createOrForceNodeOffline
bq.  
bq.  The above code also updates the inmemory state of OFFLINE or PENDING_OPEN
bq.  which was not update in the previous forceRegionStateToOffline() call.
bq.  ==============================================================================
bq.  In ZKAssign.java()
bq.      if (version == -1) {
bq.        // If timeoutmonitor deducts a node to be in OPENING state but before it
bq.        // could
bq.        // transit to OFFLINE state if RS had opened the region then the Master
bq.        // deletes the
bq.        // assigned region znode. In that case the znode will not exist. So we
bq.        // should not
bq.        // create the znode again which will lead to double assignment.
bq.        if (timeOutMonitorReAllocate && !allowCreation) {
bq.          return -1;
bq.        }
bq.  this part prevents double assignment
bq.  If timeoutmonitor tries to force to OFFLINE state an existing region which was in RIT
bq.  but before it could do that if the node was opened then openedregionhandler will delete
bq.  the node hence we should not create the node again.
bq.  
bq.  =======================================================================================
bq.  In ZkAssign.java()
bq.      } else {
bq.        RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
bq.            .getEncodedName(), stat);
bq.        // Do not move the node to OFFLINE if znode is in any of the following
bq.        // state.
bq.        // Because these are already executed states.
bq.        if (timeOutMonitorReAllocate && null != curDataInZNode) {
bq.          EventType eventType = curDataInZNode.getEventType();
bq.          if (eventType.equals(EventType.RS_ZK_REGION_CLOSING)
bq.              || eventType.equals(EventType.RS_ZK_REGION_CLOSED)
bq.              || eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
bq.            return -1;
bq.          }
bq.        }
bq.  This check prevents from moving the node to OFFLINE state if just before
bq.  the node is tried to force to OFFLINE the RS would have changed the state 
bq.  to either CLOSING or CLOSED or OPENED.  now again moving to OFFLINE
bq.  will lead to douoble assignment and will an additional operaition.
bq.  
bq.  ====================================================================================
bq.  In ZKassign.java()
bq.        boolean setData = false;
bq.        try {
bq.          setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
bq.          // Setdata throws KeeperException which aborts the Master. So we are
bq.          // catching it here.
bq.          // If just before setting the znode to OFFLINE if the RS has made any
bq.          // change to the
bq.          // znode state then we need to return -1.
bq.        } catch (KeeperException kpe) {
bq.          LOG.info("Version mismatch while setting the node to OFFLINE state.");
bq.          return -1;
bq.        }
bq.  This change is actually to avoid the master from abortng. If the forceful OFFLINE is in
bq.  progrss but just before setting the RS has changed the state.
bq.  Then the setData will fail leading to master abort.
bq.  Hence we are catching the exception.
bq.  ====================================================================================
bq.  
bq.  In assignmentManager.java
bq.  +      if (setOfflineInZK && versionOfOfflineNode == -1)
bq.  +        return;
bq.  This is nothing but the refactoring done in the existing code.
bq.  -     if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
bq.  So if setting the version is unsuccessful return.
bq.  =====================================================================================
bq.  In ZKassign.java()
bq.  // the below check ensure that double assignment doesnot happen.
bq.      // When the node is created for the first time then the expected version
bq.      // that is
bq.      // passed will be -1 and the version in znode will be 0.
bq.      // In all other cases the version in znode will be > 0.
bq.      else if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
bq.          && endState.equals(EventType.RS_ZK_REGION_OPENING)
bq.          && expectedVersion == -1 && stat.getVersion() != 0) {
bq.        LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
bq.            + encoded + " from " + beginState + " to " + endState + " failed, "
bq.            + "the node existed but was version " + stat.getVersion()
bq.            + " not the expected version " + expectedVersion));
bq.        return -1;
bq.      }
bq.  As the comment explains when the node is created for first time the expectedversion will
bq.  be -1 but the actual version will be 0.  Here the scenario is
bq.  If RS1 has not tranitioned the node from OFFLINE to OPENING if
bq.  RS2 gets the call from Master after forcefully chaning to OFFLINE the
bq.  Rs2 will take the control of the node.
bq.  At that time if RS1 starts transmitting the node then we should not allow
bq.  it.
bq.  ===============================================================================
bq.  
bq.  
bq.  This addresses bug HBASE-4015.
bq.      https://issues.apache.org/jira/browse/HBASE-4015
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java PRE-CREATION 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java 1162313 
bq.  
bq.  Diff: https://reviews.apache.org/r/1668/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  Yes.  But could not add new test case.
bq.  TestMasterFailOver is passing with the current changes also.
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  ramkrishna
bq.  
bq.



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13085535#comment-13085535 ] 

stack commented on HBASE-4015:
------------------------------

@Ram

bq. Reg timeout monitor the one major change is now the CLSOING state node is created by master itself and it was done by RS as in 0.90. Apart from this i dint find any big difference till now.

You are right I believe.

Do you think we need your proposal of 12/Aug/11 12:17 if the master passes znode version on open?

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13085841#comment-13085841 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

@Stack,
Have made the basic changes as discussed.  Will verify scenarios and will upload the patch sooner.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13098959#comment-13098959 ] 

jiraposter@reviews.apache.org commented on HBASE-4015:
------------------------------------------------------


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/1668/#review1786
-----------------------------------------------------------



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4116>

    I think preempt alone would suffice.
    We don't need to place hijack and preempt together.


- Ted


On 2011-08-29 12:11:23, ramkrishna vasudevan wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/1668/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2011-08-29 12:11:23)
bq.  
bq.  
bq.  Review request for Ted Yu, Michael Stack, Jean-Daniel Cryans, and Jonathan Gray.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  HBASE-4015 - updated patch.
bq.  
bq.  
bq.          invokeTimeOutManager(regionState.getRegion(),
bq.              TimeOutOperationType.ASSIGN);
bq.  
bq.  Instead of storing the state when timeout was deducted we process it
bq.  using future task.
bq.  ================================================================
bq.  
bq.        case RS_ZK_REGION_OPENING:
bq.          // TODO: Could check if it was on deadServers.  If it was, then we could
bq.          // do what happens in TimeoutMonitor when it sees this condition.
bq.  
bq.          // Just insert region into RIT
bq.          // If this never updates the timeout will trigger new assignment
bq.          if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
bq.            regionsInTransition.put(encodedRegionName, new RegionState(
bq.                regionInfo, RegionState.State.OPENING, data.getStamp(), data
bq.                    .getOrigin()));
bq.            processOpeningState(regionInfo);
bq.            break;
bq.          }
bq.          regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
bq.              RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
bq.          break;
bq.  
bq.  This change is for HBASE-4203. META and ROOT table need not wait till timeout
bq.  ======================================================================
bq.  
bq.  In forceRegionStateToOffline()
bq.  
bq.      } else {
bq.        // If invoked from timeout monitor donot force it to OFFLINE. Based on the
bq.        // state we will decide if to change in-memory state to OFFLINE or not.  It will
bq.        // be done before setting the znode to OFFLINE state.
bq.        if (!timeOutMonitorReAllocate) {
bq.          LOG.debug("Forcing OFFLINE; was=" + state);
bq.          state.update(RegionState.State.OFFLINE);
bq.        }
bq.  If the timeout monitor tries to reallcoate the node then dont make
bq.  the inmemory state to OFFLINE.
bq.  But the noraml assign flow doesnot expect the inmemory state to OFFLINE.
bq.  Hence the above change.  This is continued with the check in 
bq.  assign()
bq.  int setOfflineInZooKeeper(final RegionState state,
bq.        boolean timeOutMonitorReAllocate) {
bq.      // If invoked from timeoutmonitor the current state in memory need not be
bq.      // OFFLINE.  
bq.      if (!timeOutMonitorReAllocate && !state.isClosed() && !state.isOffline()) {
bq.            this.master.abort("Unexpected state trying to OFFLINE; " + state,
bq.            new IllegalStateException());
bq.        return -1;
bq.      }
bq.  ======================================================================
bq.  
bq.      boolean allowCreation = false;
bq.      // If the isReAllocate is true and the current state is PENDING_OPEN
bq.      // or OPENING then update the inmemory state to PENDING_OPEN. This is
bq.      // important because
bq.      // if timeoutmonitor deducts that a region was in OPENING state for a long
bq.      // time but by the
bq.      // time timeout monitor tranits the node to OFFLINE the RS would have opened
bq.      // the node and the
bq.      // state in znode will be RS_ZK_REGION_OPENED. Inorder to invoke the
bq.      // OpenedRegionHandler
bq.      // we expect the inmemeory state to be PENDING_OPEN or OPENING.
bq.      // For all other cases we can change the inmemory state to OFFLINE.
bq.      if (timeOutMonitorReAllocate
bq.          && (state.getState().equals(RegionState.State.PENDING_OPEN) || state
bq.              .getState().equals(RegionState.State.OPENING))) {
bq.        state.update(RegionState.State.PENDING_OPEN);
bq.        allowCreation = false;
bq.      } else {
bq.        state.update(RegionState.State.OFFLINE);
bq.        allowCreation = true;
bq.      }
bq.  This change is quite tricky.  
bq.  In normal assign flow the unassigned node for the region will not be present
bq.  Hence we need to allow the creation of the node newly.
bq.  But in timeout monitor case we will have the node present in some state hence 
bq.  we decide whether to create node newly or not inside ZKAssign.createOrForceNodeOffline
bq.  
bq.  The above code also updates the inmemory state of OFFLINE or PENDING_OPEN
bq.  which was not update in the previous forceRegionStateToOffline() call.
bq.  ==============================================================================
bq.  In ZKAssign.java()
bq.      if (version == -1) {
bq.        // If timeoutmonitor deducts a node to be in OPENING state but before it
bq.        // could
bq.        // transit to OFFLINE state if RS had opened the region then the Master
bq.        // deletes the
bq.        // assigned region znode. In that case the znode will not exist. So we
bq.        // should not
bq.        // create the znode again which will lead to double assignment.
bq.        if (timeOutMonitorReAllocate && !allowCreation) {
bq.          return -1;
bq.        }
bq.  this part prevents double assignment
bq.  If timeoutmonitor tries to force to OFFLINE state an existing region which was in RIT
bq.  but before it could do that if the node was opened then openedregionhandler will delete
bq.  the node hence we should not create the node again.
bq.  
bq.  =======================================================================================
bq.  In ZkAssign.java()
bq.      } else {
bq.        RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
bq.            .getEncodedName(), stat);
bq.        // Do not move the node to OFFLINE if znode is in any of the following
bq.        // state.
bq.        // Because these are already executed states.
bq.        if (timeOutMonitorReAllocate && null != curDataInZNode) {
bq.          EventType eventType = curDataInZNode.getEventType();
bq.          if (eventType.equals(EventType.RS_ZK_REGION_CLOSING)
bq.              || eventType.equals(EventType.RS_ZK_REGION_CLOSED)
bq.              || eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
bq.            return -1;
bq.          }
bq.        }
bq.  This check prevents from moving the node to OFFLINE state if just before
bq.  the node is tried to force to OFFLINE the RS would have changed the state 
bq.  to either CLOSING or CLOSED or OPENED.  now again moving to OFFLINE
bq.  will lead to douoble assignment and will an additional operaition.
bq.  
bq.  ====================================================================================
bq.  In ZKassign.java()
bq.        boolean setData = false;
bq.        try {
bq.          setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
bq.          // Setdata throws KeeperException which aborts the Master. So we are
bq.          // catching it here.
bq.          // If just before setting the znode to OFFLINE if the RS has made any
bq.          // change to the
bq.          // znode state then we need to return -1.
bq.        } catch (KeeperException kpe) {
bq.          LOG.info("Version mismatch while setting the node to OFFLINE state.");
bq.          return -1;
bq.        }
bq.  This change is actually to avoid the master from abortng. If the forceful OFFLINE is in
bq.  progrss but just before setting the RS has changed the state.
bq.  Then the setData will fail leading to master abort.
bq.  Hence we are catching the exception.
bq.  ====================================================================================
bq.  
bq.  In assignmentManager.java
bq.  +      if (setOfflineInZK && versionOfOfflineNode == -1)
bq.  +        return;
bq.  This is nothing but the refactoring done in the existing code.
bq.  -     if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
bq.  So if setting the version is unsuccessful return.
bq.  =====================================================================================
bq.  In ZKassign.java()
bq.  // the below check ensure that double assignment doesnot happen.
bq.      // When the node is created for the first time then the expected version
bq.      // that is
bq.      // passed will be -1 and the version in znode will be 0.
bq.      // In all other cases the version in znode will be > 0.
bq.      else if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
bq.          && endState.equals(EventType.RS_ZK_REGION_OPENING)
bq.          && expectedVersion == -1 && stat.getVersion() != 0) {
bq.        LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
bq.            + encoded + " from " + beginState + " to " + endState + " failed, "
bq.            + "the node existed but was version " + stat.getVersion()
bq.            + " not the expected version " + expectedVersion));
bq.        return -1;
bq.      }
bq.  As the comment explains when the node is created for first time the expectedversion will
bq.  be -1 but the actual version will be 0.  Here the scenario is
bq.  If RS1 has not tranitioned the node from OFFLINE to OPENING if
bq.  RS2 gets the call from Master after forcefully chaning to OFFLINE the
bq.  Rs2 will take the control of the node.
bq.  At that time if RS1 starts transmitting the node then we should not allow
bq.  it.
bq.  ===============================================================================
bq.  
bq.  
bq.  This addresses bug HBASE-4015.
bq.      https://issues.apache.org/jira/browse/HBASE-4015
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java PRE-CREATION 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java 1161985 
bq.  
bq.  Diff: https://reviews.apache.org/r/1668/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  Yes.  But could not add new test case.
bq.  TestMasterFailOver is passing with the current changes also.
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  ramkrishna
bq.  
bq.



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Jean-Daniel Cryans (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13100438#comment-13100438 ] 

Jean-Daniel Cryans commented on HBASE-4015:
-------------------------------------------

Perfect!

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

ramkrishna.s.vasudevan updated HBASE-4015:
------------------------------------------

    Attachment: Timeoutmonitor with state diagrams.pdf

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13092313#comment-13092313 ] 

Ted Yu commented on HBASE-4015:
-------------------------------

@Ramkrishna:
If you have access to a real cluster, can you do the following:
1. create a table and load it with data so that it has over 400 regions
2. continue writing/reading the table and trigger region splits
3. issue balancer command
You can instrument TimeoutMonitor to record stats for occurrences of timeouts
At the end of test, use hbck to check consistency

Thanks

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "Ted Yu (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13090056#comment-13090056 ] 

Ted Yu commented on HBASE-4015:
-------------------------------

There are 4 files which patch-2 didn't cleanly apply on.
Producing an updated patch based on version 2 would help reviewers navigate the changes.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13096126#comment-13096126 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

The following tests were performed

The cluster has 3 nodes.
Created some 4000+ regions.  And i followed the startegy like

Initially create regions

Balancer will balance the regions across all the three.

Using a script kill the RS in these 3 nodes at fixed time.(at different time intervals).

Then start it after sometime.

Sometimes once all the 3 RS are up invoke the balancer command manually.

While balancing kill one of the RS which was the destination during balancer calculation.
Again bring it up.

The number of regions remained constant and every time the Timeout detected it was able to overcome it with no issues at all.

I also calculated some metrics

There were instance were 144, 249,1334, 30 regions were timed out.
For any timeout region to be monitored and completed it took at the max 1.5 sec minimum being 0.35 secs.

The following logs shows the scenario where the timeout monitor tries to assign a node to OFFLINE but by the time the node is changed to OPENED by RS.
{code}
2011-09-02 18:15:55,621 INFO org.apache.hadoop.hbase.master.AssignmentManager: Region has been PENDING_OPEN for too long, reassigning region=testram2,9970,1314962425951.b7a45cf31651d5cbba682ae24b83bf88.
2011-09-02 18:15:55,709 INFO org.mortbay.log: Timeout detected for the region testram2,9970,1314962425951.b7a45cf31651d5cbba682ae24b83bf88. at Fri Sep 02 18:15:55 IST 2011
2011-09-02 18:15:55,942 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:60000-0x13229c29f830010 Creating (or updating) unassigned node for b7a45cf31651d5cbba682ae24b83bf88 with OFFLINE state


2011-09-02 18:15:55,967 INFO org.apache.hadoop.hbase.zookeeper.ZKAssign: Current data in znode .region=testram2,9970,1314962425951.b7a45cf31651d5cbba682ae24b83bf88., origin=HOST-10-18-52-198,60020,1314967317396, state=RS_ZK_REGION_OPENED
2011-09-02 18:15:55,996 WARN org.apache.hadoop.hbase.master.AssignmentManager: Attempted to create/force node into OFFLINE state before completing assignment but failed to do so for testram2,9970,1314962425951.b7a45cf31651d5cbba682ae24b83bf88. state=PENDING_OPEN, ts=1314967555942, server=null

 


2011-09-02 18:15:55,999 INFO org.mortbay.log: Time taken to complete the action on timeout for the reigon testram2,9970,1314962425951.b7a45cf31651d5cbba682ae24b83bf88.291
2011-09-02 18:16:24,115 DEBUG org.apache.hadoop.hbase.master.AssignmentManager: Handling transition=RS_ZK_REGION_OPENED, server=HOST-10-18-52-198,60020,1314967317396, region=b7a45cf31651d5cbba682ae24b83bf88, which is more than 15 seconds late
2011-09-02 18:16:24,122 DEBUG org.apache.hadoop.hbase.zookeeper.ZKAssign: master:60000-0x13229c29f830010 Successfully deleted unassigned node for region b7a45cf31651d5cbba682ae24b83bf88 in expected state RS_ZK_REGION_OPENED
{code}

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13099224#comment-13099224 ] 

jiraposter@reviews.apache.org commented on HBASE-4015:
------------------------------------------------------


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/1668/#review1798
-----------------------------------------------------------

Ship it!


I think this patch at a minimum narrows the races significantly, enough to warrant application.  Writing a test is a little tough.  If we could standup an AssignmentManager outside of its hosting Master context, that'd help.  Then we could set the AM and AM.TimeoutManager in competition over znode states.  This is something to do but I'd say in another issue.

I'd say we should let this issue hang out there another 24 hours in case anyone else has an opinion before we go ahead commit.

- Michael


On 2011-09-07 15:52:26, ramkrishna vasudevan wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/1668/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2011-09-07 15:52:26)
bq.  
bq.  
bq.  Review request for Ted Yu, Michael Stack, Jean-Daniel Cryans, and Jonathan Gray.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  HBASE-4015 - updated patch.
bq.  
bq.  
bq.          invokeTimeOutManager(regionState.getRegion(),
bq.              TimeOutOperationType.ASSIGN);
bq.  
bq.  Instead of storing the state when timeout was deducted we process it
bq.  using future task.
bq.  ================================================================
bq.  
bq.        case RS_ZK_REGION_OPENING:
bq.          // TODO: Could check if it was on deadServers.  If it was, then we could
bq.          // do what happens in TimeoutMonitor when it sees this condition.
bq.  
bq.          // Just insert region into RIT
bq.          // If this never updates the timeout will trigger new assignment
bq.          if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
bq.            regionsInTransition.put(encodedRegionName, new RegionState(
bq.                regionInfo, RegionState.State.OPENING, data.getStamp(), data
bq.                    .getOrigin()));
bq.            processOpeningState(regionInfo);
bq.            break;
bq.          }
bq.          regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
bq.              RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
bq.          break;
bq.  
bq.  This change is for HBASE-4203. META and ROOT table need not wait till timeout
bq.  ======================================================================
bq.  
bq.  In forceRegionStateToOffline()
bq.  
bq.      } else {
bq.        // If invoked from timeout monitor donot force it to OFFLINE. Based on the
bq.        // state we will decide if to change in-memory state to OFFLINE or not.  It will
bq.        // be done before setting the znode to OFFLINE state.
bq.        if (!timeOutMonitorReAllocate) {
bq.          LOG.debug("Forcing OFFLINE; was=" + state);
bq.          state.update(RegionState.State.OFFLINE);
bq.        }
bq.  If the timeout monitor tries to reallcoate the node then dont make
bq.  the inmemory state to OFFLINE.
bq.  But the noraml assign flow doesnot expect the inmemory state to OFFLINE.
bq.  Hence the above change.  This is continued with the check in 
bq.  assign()
bq.  int setOfflineInZooKeeper(final RegionState state,
bq.        boolean timeOutMonitorReAllocate) {
bq.      // If invoked from timeoutmonitor the current state in memory need not be
bq.      // OFFLINE.  
bq.      if (!timeOutMonitorReAllocate && !state.isClosed() && !state.isOffline()) {
bq.            this.master.abort("Unexpected state trying to OFFLINE; " + state,
bq.            new IllegalStateException());
bq.        return -1;
bq.      }
bq.  ======================================================================
bq.  
bq.      boolean allowCreation = false;
bq.      // If the isReAllocate is true and the current state is PENDING_OPEN
bq.      // or OPENING then update the inmemory state to PENDING_OPEN. This is
bq.      // important because
bq.      // if timeoutmonitor deducts that a region was in OPENING state for a long
bq.      // time but by the
bq.      // time timeout monitor tranits the node to OFFLINE the RS would have opened
bq.      // the node and the
bq.      // state in znode will be RS_ZK_REGION_OPENED. Inorder to invoke the
bq.      // OpenedRegionHandler
bq.      // we expect the inmemeory state to be PENDING_OPEN or OPENING.
bq.      // For all other cases we can change the inmemory state to OFFLINE.
bq.      if (timeOutMonitorReAllocate
bq.          && (state.getState().equals(RegionState.State.PENDING_OPEN) || state
bq.              .getState().equals(RegionState.State.OPENING))) {
bq.        state.update(RegionState.State.PENDING_OPEN);
bq.        allowCreation = false;
bq.      } else {
bq.        state.update(RegionState.State.OFFLINE);
bq.        allowCreation = true;
bq.      }
bq.  This change is quite tricky.  
bq.  In normal assign flow the unassigned node for the region will not be present
bq.  Hence we need to allow the creation of the node newly.
bq.  But in timeout monitor case we will have the node present in some state hence 
bq.  we decide whether to create node newly or not inside ZKAssign.createOrForceNodeOffline
bq.  
bq.  The above code also updates the inmemory state of OFFLINE or PENDING_OPEN
bq.  which was not update in the previous forceRegionStateToOffline() call.
bq.  ==============================================================================
bq.  In ZKAssign.java()
bq.      if (version == -1) {
bq.        // If timeoutmonitor deducts a node to be in OPENING state but before it
bq.        // could
bq.        // transit to OFFLINE state if RS had opened the region then the Master
bq.        // deletes the
bq.        // assigned region znode. In that case the znode will not exist. So we
bq.        // should not
bq.        // create the znode again which will lead to double assignment.
bq.        if (timeOutMonitorReAllocate && !allowCreation) {
bq.          return -1;
bq.        }
bq.  this part prevents double assignment
bq.  If timeoutmonitor tries to force to OFFLINE state an existing region which was in RIT
bq.  but before it could do that if the node was opened then openedregionhandler will delete
bq.  the node hence we should not create the node again.
bq.  
bq.  =======================================================================================
bq.  In ZkAssign.java()
bq.      } else {
bq.        RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
bq.            .getEncodedName(), stat);
bq.        // Do not move the node to OFFLINE if znode is in any of the following
bq.        // state.
bq.        // Because these are already executed states.
bq.        if (timeOutMonitorReAllocate && null != curDataInZNode) {
bq.          EventType eventType = curDataInZNode.getEventType();
bq.          if (eventType.equals(EventType.RS_ZK_REGION_CLOSING)
bq.              || eventType.equals(EventType.RS_ZK_REGION_CLOSED)
bq.              || eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
bq.            return -1;
bq.          }
bq.        }
bq.  This check prevents from moving the node to OFFLINE state if just before
bq.  the node is tried to force to OFFLINE the RS would have changed the state 
bq.  to either CLOSING or CLOSED or OPENED.  now again moving to OFFLINE
bq.  will lead to douoble assignment and will an additional operaition.
bq.  
bq.  ====================================================================================
bq.  In ZKassign.java()
bq.        boolean setData = false;
bq.        try {
bq.          setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
bq.          // Setdata throws KeeperException which aborts the Master. So we are
bq.          // catching it here.
bq.          // If just before setting the znode to OFFLINE if the RS has made any
bq.          // change to the
bq.          // znode state then we need to return -1.
bq.        } catch (KeeperException kpe) {
bq.          LOG.info("Version mismatch while setting the node to OFFLINE state.");
bq.          return -1;
bq.        }
bq.  This change is actually to avoid the master from abortng. If the forceful OFFLINE is in
bq.  progrss but just before setting the RS has changed the state.
bq.  Then the setData will fail leading to master abort.
bq.  Hence we are catching the exception.
bq.  ====================================================================================
bq.  
bq.  In assignmentManager.java
bq.  +      if (setOfflineInZK && versionOfOfflineNode == -1)
bq.  +        return;
bq.  This is nothing but the refactoring done in the existing code.
bq.  -     if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
bq.  So if setting the version is unsuccessful return.
bq.  =====================================================================================
bq.  In ZKassign.java()
bq.  // the below check ensure that double assignment doesnot happen.
bq.      // When the node is created for the first time then the expected version
bq.      // that is
bq.      // passed will be -1 and the version in znode will be 0.
bq.      // In all other cases the version in znode will be > 0.
bq.      else if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
bq.          && endState.equals(EventType.RS_ZK_REGION_OPENING)
bq.          && expectedVersion == -1 && stat.getVersion() != 0) {
bq.        LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
bq.            + encoded + " from " + beginState + " to " + endState + " failed, "
bq.            + "the node existed but was version " + stat.getVersion()
bq.            + " not the expected version " + expectedVersion));
bq.        return -1;
bq.      }
bq.  As the comment explains when the node is created for first time the expectedversion will
bq.  be -1 but the actual version will be 0.  Here the scenario is
bq.  If RS1 has not tranitioned the node from OFFLINE to OPENING if
bq.  RS2 gets the call from Master after forcefully chaning to OFFLINE the
bq.  Rs2 will take the control of the node.
bq.  At that time if RS1 starts transmitting the node then we should not allow
bq.  it.
bq.  ===============================================================================
bq.  
bq.  
bq.  This addresses bug HBASE-4015.
bq.      https://issues.apache.org/jira/browse/HBASE-4015
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 1166224 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignCallable.java PRE-CREATION 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java 1166224 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 1166224 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1166224 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/UnAssignCallable.java PRE-CREATION 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1166224 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java 1166224 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java 1166224 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java 1166224 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java 1166224 
bq.  
bq.  Diff: https://reviews.apache.org/r/1668/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  Yes.  But could not add new test case.
bq.  TestMasterFailOver is passing with the current changes also.
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  ramkrishna
bq.  
bq.



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13090021#comment-13090021 ] 

stack commented on HBASE-4015:
------------------------------

@Ted In most other state transitions, we have the znode version to hand (we previously read the znode state).  Its this one operation where we are transferring region ownership from master to regionserver that has the hole.  A new state "seems" to be a patch on the actual problem but I could be convinced otherwise (Looking at Ram's diagram the new state seemed cleaner but I had sneaking suspicion that the new state introduces a bunch of new conditions not dealt with in the diagram).

On the patch (Thanks for digging in here Ram):

Why the special handling of meta regions here:

{code}
-        regionsInTransition.put(encodedRegionName, new RegionState(
-            regionInfo, RegionState.State.OPENING,
-            data.getStamp(), data.getOrigin()));
+        if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
+          regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
+              RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
+          processOpeningState(regionInfo);
+          break;
+        }
+        regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
+            RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
{code}

FYI, IIRC, regionInfo.isMetaRegion() is true if .META. or -ROOT-

This is ok, not setting a watcher here?

{code}
+      RegionTransitionData dataInZNode = ZKAssign.getDataNoWatch(watcher, node,
+          stat);
{code}

Here you might want to add logging whether or not its a reallocate:

{code}
-      LOG.info("Table " + tableName + (disabled? " disabled;": " disabling;") +
-        " skipping assign of " + region.getRegionNameAsString());
+      LOG.info("Table " + tableName + (disabled ? " disabled;" : " disabling;")
+          + " skipping assign of " + region.getRegionNameAsString());
{code}

FYI, I prefer the previous style where the '+' is at the end of the line; it flags reader that there is more to come.... (IMO).

What is happening here?  Who will have set realloc true?  The timeout monitor?  So who then will set it to OFFLINE?

{code}
-      LOG.debug("Forcing OFFLINE; was=" + state);
-      state.update(RegionState.State.OFFLINE);
+      // If invoked from timeout monitor donot force it to OFFLINE. Based on the
+      // state
+      // we will decide if to change in-memory state to OFFLINE or not.
+      if (!isReAllocate) {
+        LOG.debug("Forcing OFFLINE; was=" + state);
+        state.update(RegionState.State.OFFLINE);
+      }
{code}

When would this happen:

{code}
+      if (setOfflineInZK && versionOfOfflineNode == -1)
+        return;
{code}

i.e... it would come back w/ -1?

Style: If multiple lines, it needs curly brackets (Would suggest leaving stuff like this alone)

{code}
-      if (plan == null) return; // Should get reassigned later when RIT times out.
+      if (plan == null)
+        return; // Should get reassigned later when RIT times out.
{code}

At the other extreme, its fine to fix stuff like this where its w/o spaces around the '+' operator:

{code}
-            if(LOG.isDebugEnabled()){
-              LOG.debug("The unassigned node "+encodedRegionName+" doesnot exist.");
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("The unassigned node " + encodedRegionName
+                  + " doesnot exist.");
{code}

(though you could separate the 'doesnot' into 'does not'

This kinda change just bloats your patch (making Ted think it does more than it does -- smile):

{code}
-        LOG.warn("Failed assignment of " +
-          state.getRegion().getRegionNameAsString() + " to " +
-          plan.getDestination() + ", trying to assign elsewhere instead; " +
-          "retry=" + i, t);
+        LOG.warn("Failed assignment of "
+            + state.getRegion().getRegionNameAsString() + " to "
+            + plan.getDestination() + ", trying to assign elsewhere instead; "
+            + "retry=" + i, t);
{code}

ditto

{code}
-          LOG.warn("Unable to find a viable location to assign region " +
-            state.getRegion().getRegionNameAsString());
+          LOG.warn("Unable to find a viable location to assign region "
+              + state.getRegion().getRegionNameAsString());
{code}

Does the param 'isReAllocate' mean method was called from timeout monitor?  If so, it probably warrants a different name?

What is going to happen if we return -1 because of below?

{code}
+      if (versionOfOfflineNode == -1) {
+        LOG.warn("Attempted to create/force node into OFFLINE state before "
+            + "completing assignment but failed to do so for " + state);
+        return -1;
       }
{code}

+1 on refactor of timeout breaking out the method actOnTimeout.

Did you change anything in way timeout monitor works?

This is interesting:

{code}
+  public static enum TimeOutOperationType {
+    ASSIGN, UNASSIGN;
+  }
{code}

We have timeout types.  How does this work?  Is this an in-memory master state only?  What happens if master crashes?  Will new master need to know these two new states?

Why are we passing AssignmentManager a threadPoolExecutorService?  Its used by timeout monitor?  Should creation of the executor service just live in the timeout monitor and be constructed there rather than up in master and passed in?  Or is it that you need to be able to shut it down when master goes down?

Add a comment on TimeOutManagerCallable class saying what its for.

This is great:

{code}
+      if (!transitionZookeeperOfflineToOpening(encodedName,
+          versionOfOfflineNode)) {
{code}

... over on the RS.

Is this code copy paste from the method above it?  The version that does not take a znode version?

{code}
+  public RegionOpeningState openRegion(HRegionInfo region, int expectedVersion)
{code}

If so, we should fix that.

This patch needs closer review and I'd like to know how this executor service in timeout monitor solve races we've seen (it move operations out of line w/ processing of the actual timeout event?) but I'm thinking this patch looks good; just what the doctor ordered, at least around passing of the version.  What are your reservations Ram?  How hard would it be to add tests?

Good stuff.



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13091798#comment-13091798 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

bq.Why are we passing AssignmentManager a threadPoolExecutorService? Its used by timeout monitor? Should creation of the executor service just live in the timeout monitor and be constructed there rather than up in master and passed in? Or is it that you need to be able to shut it down when master goes down?
Yes it is to shutdown the threadpoolexecutor service.
bq.+  public static enum TimeOutOperationType {
+    ASSIGN, UNASSIGN;
+  }
This is only an indication for the timeoutmanager callable to know whether to assign or unassign 

Formatting issues in previous patch has also been taken care.

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13092285#comment-13092285 ] 

jiraposter@reviews.apache.org commented on HBASE-4015:
------------------------------------------------------



bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > Looks great Ram.  WOuld be hard to write a test for this I think.  What do you think?

Yes Stack.  I tried but could not. If you can give some suggestions i could do that.


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java, line 345
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35892#file35892line345>
bq.  >
bq.  >     White space.

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java, line 349
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35892#file35892line349>
bq.  >
bq.  >     Is it the RS that is creating the OFFLINE node or is it the master that does this and RS moves the region from OFFLINE to OPENING?  No need to say 'passed by the master'

updated the comment


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java, line 351
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35892#file35892line351>
bq.  >
bq.  >     Is this correct?  The open is async w/ the states of opening done up in zk w/ master noticing the last -- right?  The return is whether or not the request to open was accepted?

updated the comment.


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java, line 354
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35892#file35892line354>
bq.  >
bq.  >     ws

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java, line 359
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35892#file35892line359>
bq.  >
bq.  >     ws

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 148
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line148>
bq.  >
bq.  >     ws

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 160
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line160>
bq.  >
bq.  >     ws

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 410
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line410>
bq.  >
bq.  >     So, whats this special-handling of catalog tables about?  Want to add comment why it is so?

added suitable comment.  This is for HBASE-4203


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1034
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line1034>
bq.  >
bq.  >     Todd would say don't bother adding javadoc if it says nothing other than repeat param names.

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1045
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line1045>
bq.  >
bq.  >     This is a better name for this param though why does this method need to know who is calling it?  Perhaps there is a better name?  timedout?  What you think Ram?
bq.  >     
bq.  >     Do you mean 'reassign' when you say 'reallocate'
bq.  >     
bq.  >     So maybe use 'reassign' instead of 'timedout'?

renamed to reassign. And only in the calling method i have docuemented about this variable.


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1235
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line1235>
bq.  >
bq.  >     ws

removed.


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1249
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line1249>
bq.  >
bq.  >     ws

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1264
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line1264>
bq.  >
bq.  >     When do we do this?  Where do we do the change of in-memory state?

added suitable comment.
Actually the original state is updated just before making the node to OFFLINE state.
It is done in setOfflineInZooKeeper()


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1277
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line1277>
bq.  >
bq.  >     ws

remove


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1281
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line1281>
bq.  >
bq.  >     yeah, why should this method care where its called from so would suggest removing timeoutMonitor from main and name param for condition under which this method is being called.

changed to reassign


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1299
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line1299>
bq.  >
bq.  >     ws

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1304
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line1304>
bq.  >
bq.  >     ws

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1380
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line1380>
bq.  >
bq.  >     ws

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1390
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line1390>
bq.  >
bq.  >     ws

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1396
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line1396>
bq.  >
bq.  >     maybe 'allowZNodeCreation' instead?

done


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1400
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line1400>
bq.  >
bq.  >     When you say 'deducts' do you mean to use the current tense?  If so, that'd be 'deduces'.

changed to deduces.


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1407
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line1407>
bq.  >
bq.  >     Can you fix up this comment so it aligns a bit better?
bq.  >     
bq.  >     There is some misspelling in this paragraph that you might want to fix?

updated the comment. Pls let me know if it is fine.


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1408
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line1408>
bq.  >
bq.  >     Can you format this a little better?  Have first line end with && and then second line end with ||.  The state.getState....should be on a single line ... make it easier to read rather than split over two lines.

done


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 1424
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line1424>
bq.  >
bq.  >     So, what you think will happen next?  We'll fail the assign?  Region will be stuck?

Stack, this step is the same as in the previous code.
I think if the state is not changed agin Timeout will try to assign it.
-      if(!ZKAssign.createOrForceNodeOffline(master.getZooKeeper(),
-          state.getRegion(), this.master.getServerName())) {
-        LOG.warn("Attempted to create/force node into OFFLINE state before " +
-          "completing assignment but failed to do so for " + state);
-        return false;


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 2351
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line2351>
bq.  >
bq.  >     FYI, I prefer the former formatting where the arg was all on one line rather than split over two.

done


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 2386
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line2386>
bq.  >
bq.  >     ws

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java, line 185
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35894#file35894line185>
bq.  >
bq.  >     ws

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java, line 58
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35896#file35896line58>
bq.  >
bq.  >     why extra lines down here?

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java, line 2329
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35897#file35897line2329>
bq.  >
bq.  >     ws

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java, line 55
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35899#file35899line55>
bq.  >
bq.  >     ws

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java, line 236
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35901#file35901line236>
bq.  >
bq.  >     ws

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java, line 761
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35901#file35901line761>
bq.  >
bq.  >     ws

removed


bq.  On 2011-08-27 00:20:23, Michael Stack wrote:
bq.  > http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java, line 2423
bq.  > <https://reviews.apache.org/r/1668/diff/1/?file=35893#file35893line2423>
bq.  >
bq.  >     Did you change these Excecutors to verify that the state that caused their being queued still holds before they actually run?  That would be important I'd think if you have not done it that is.

Yes stack just before we change the allow the new assignment to start the states has been
verified and only then we have proceeded with forcing the node to OFFLINE once again.


- ramkrishna


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/1668/#review1666
-----------------------------------------------------------


On 2011-08-27 11:51:16, ramkrishna vasudevan wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/1668/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2011-08-27 11:51:16)
bq.  
bq.  
bq.  Review request for Ted Yu, Michael Stack, Jean-Daniel Cryans, and Jonathan Gray.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  HBASE-4015 - updated patch.
bq.  
bq.  
bq.          invokeTimeOutManager(regionState.getRegion(),
bq.              TimeOutOperationType.ASSIGN);
bq.  
bq.  Instead of storing the state when timeout was deducted we process it
bq.  using future task.
bq.  ================================================================
bq.  
bq.        case RS_ZK_REGION_OPENING:
bq.          // TODO: Could check if it was on deadServers.  If it was, then we could
bq.          // do what happens in TimeoutMonitor when it sees this condition.
bq.  
bq.          // Just insert region into RIT
bq.          // If this never updates the timeout will trigger new assignment
bq.          if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
bq.            regionsInTransition.put(encodedRegionName, new RegionState(
bq.                regionInfo, RegionState.State.OPENING, data.getStamp(), data
bq.                    .getOrigin()));
bq.            processOpeningState(regionInfo);
bq.            break;
bq.          }
bq.          regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
bq.              RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
bq.          break;
bq.  
bq.  This change is for HBASE-4203. META and ROOT table need not wait till timeout
bq.  ======================================================================
bq.  
bq.  In forceRegionStateToOffline()
bq.  
bq.      } else {
bq.        // If invoked from timeout monitor donot force it to OFFLINE. Based on the
bq.        // state we will decide if to change in-memory state to OFFLINE or not.  It will
bq.        // be done before setting the znode to OFFLINE state.
bq.        if (!timeOutMonitorReAllocate) {
bq.          LOG.debug("Forcing OFFLINE; was=" + state);
bq.          state.update(RegionState.State.OFFLINE);
bq.        }
bq.  If the timeout monitor tries to reallcoate the node then dont make
bq.  the inmemory state to OFFLINE.
bq.  But the noraml assign flow doesnot expect the inmemory state to OFFLINE.
bq.  Hence the above change.  This is continued with the check in 
bq.  assign()
bq.  int setOfflineInZooKeeper(final RegionState state,
bq.        boolean timeOutMonitorReAllocate) {
bq.      // If invoked from timeoutmonitor the current state in memory need not be
bq.      // OFFLINE.  
bq.      if (!timeOutMonitorReAllocate && !state.isClosed() && !state.isOffline()) {
bq.            this.master.abort("Unexpected state trying to OFFLINE; " + state,
bq.            new IllegalStateException());
bq.        return -1;
bq.      }
bq.  ======================================================================
bq.  
bq.      boolean allowCreation = false;
bq.      // If the isReAllocate is true and the current state is PENDING_OPEN
bq.      // or OPENING then update the inmemory state to PENDING_OPEN. This is
bq.      // important because
bq.      // if timeoutmonitor deducts that a region was in OPENING state for a long
bq.      // time but by the
bq.      // time timeout monitor tranits the node to OFFLINE the RS would have opened
bq.      // the node and the
bq.      // state in znode will be RS_ZK_REGION_OPENED. Inorder to invoke the
bq.      // OpenedRegionHandler
bq.      // we expect the inmemeory state to be PENDING_OPEN or OPENING.
bq.      // For all other cases we can change the inmemory state to OFFLINE.
bq.      if (timeOutMonitorReAllocate
bq.          && (state.getState().equals(RegionState.State.PENDING_OPEN) || state
bq.              .getState().equals(RegionState.State.OPENING))) {
bq.        state.update(RegionState.State.PENDING_OPEN);
bq.        allowCreation = false;
bq.      } else {
bq.        state.update(RegionState.State.OFFLINE);
bq.        allowCreation = true;
bq.      }
bq.  This change is quite tricky.  
bq.  In normal assign flow the unassigned node for the region will not be present
bq.  Hence we need to allow the creation of the node newly.
bq.  But in timeout monitor case we will have the node present in some state hence 
bq.  we decide whether to create node newly or not inside ZKAssign.createOrForceNodeOffline
bq.  
bq.  The above code also updates the inmemory state of OFFLINE or PENDING_OPEN
bq.  which was not update in the previous forceRegionStateToOffline() call.
bq.  ==============================================================================
bq.  In ZKAssign.java()
bq.      if (version == -1) {
bq.        // If timeoutmonitor deducts a node to be in OPENING state but before it
bq.        // could
bq.        // transit to OFFLINE state if RS had opened the region then the Master
bq.        // deletes the
bq.        // assigned region znode. In that case the znode will not exist. So we
bq.        // should not
bq.        // create the znode again which will lead to double assignment.
bq.        if (timeOutMonitorReAllocate && !allowCreation) {
bq.          return -1;
bq.        }
bq.  this part prevents double assignment
bq.  If timeoutmonitor tries to force to OFFLINE state an existing region which was in RIT
bq.  but before it could do that if the node was opened then openedregionhandler will delete
bq.  the node hence we should not create the node again.
bq.  
bq.  =======================================================================================
bq.  In ZkAssign.java()
bq.      } else {
bq.        RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
bq.            .getEncodedName(), stat);
bq.        // Do not move the node to OFFLINE if znode is in any of the following
bq.        // state.
bq.        // Because these are already executed states.
bq.        if (timeOutMonitorReAllocate && null != curDataInZNode) {
bq.          EventType eventType = curDataInZNode.getEventType();
bq.          if (eventType.equals(EventType.RS_ZK_REGION_CLOSING)
bq.              || eventType.equals(EventType.RS_ZK_REGION_CLOSED)
bq.              || eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
bq.            return -1;
bq.          }
bq.        }
bq.  This check prevents from moving the node to OFFLINE state if just before
bq.  the node is tried to force to OFFLINE the RS would have changed the state 
bq.  to either CLOSING or CLOSED or OPENED.  now again moving to OFFLINE
bq.  will lead to douoble assignment and will an additional operaition.
bq.  
bq.  ====================================================================================
bq.  In ZKassign.java()
bq.        boolean setData = false;
bq.        try {
bq.          setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
bq.          // Setdata throws KeeperException which aborts the Master. So we are
bq.          // catching it here.
bq.          // If just before setting the znode to OFFLINE if the RS has made any
bq.          // change to the
bq.          // znode state then we need to return -1.
bq.        } catch (KeeperException kpe) {
bq.          LOG.info("Version mismatch while setting the node to OFFLINE state.");
bq.          return -1;
bq.        }
bq.  This change is actually to avoid the master from abortng. If the forceful OFFLINE is in
bq.  progrss but just before setting the RS has changed the state.
bq.  Then the setData will fail leading to master abort.
bq.  Hence we are catching the exception.
bq.  ====================================================================================
bq.  
bq.  In assignmentManager.java
bq.  +      if (setOfflineInZK && versionOfOfflineNode == -1)
bq.  +        return;
bq.  This is nothing but the refactoring done in the existing code.
bq.  -     if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
bq.  So if setting the version is unsuccessful return.
bq.  =====================================================================================
bq.  In ZKassign.java()
bq.  // the below check ensure that double assignment doesnot happen.
bq.      // When the node is created for the first time then the expected version
bq.      // that is
bq.      // passed will be -1 and the version in znode will be 0.
bq.      // In all other cases the version in znode will be > 0.
bq.      else if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
bq.          && endState.equals(EventType.RS_ZK_REGION_OPENING)
bq.          && expectedVersion == -1 && stat.getVersion() != 0) {
bq.        LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
bq.            + encoded + " from " + beginState + " to " + endState + " failed, "
bq.            + "the node existed but was version " + stat.getVersion()
bq.            + " not the expected version " + expectedVersion));
bq.        return -1;
bq.      }
bq.  As the comment explains when the node is created for first time the expectedversion will
bq.  be -1 but the actual version will be 0.  Here the scenario is
bq.  If RS1 has not tranitioned the node from OFFLINE to OPENING if
bq.  RS2 gets the call from Master after forcefully chaning to OFFLINE the
bq.  Rs2 will take the control of the node.
bq.  At that time if RS1 starts transmitting the node then we should not allow
bq.  it.
bq.  ===============================================================================
bq.  
bq.  
bq.  This addresses bug HBASE-4015.
bq.      https://issues.apache.org/jira/browse/HBASE-4015
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java PRE-CREATION 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java 1162313 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java 1162313 
bq.  
bq.  Diff: https://reviews.apache.org/r/1668/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  Yes.  But could not add new test case.
bq.  TestMasterFailOver is passing with the current changes also.
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  ramkrishna
bq.  
bq.



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13092648#comment-13092648 ] 

ramkrishna.s.vasudevan commented on HBASE-4015:
-----------------------------------------------

@Ted
I will do the testing as you have told. Thanks

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "jiraposter@reviews.apache.org (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13098605#comment-13098605 ] 

jiraposter@reviews.apache.org commented on HBASE-4015:
------------------------------------------------------


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/1668/#review1778
-----------------------------------------------------------


I have some feedback below.  On how to write a test, I'm not sure how -- so many systems are touched.  We'd need to break out AM so it standalone but that is too much to ask of this patch.  Good work Ram.  Thanks for persevering with this one.


http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4056>

    I see.  You have full path here to distingush from o.a.h.h.executor.ExecutorService.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4057>

    I wonder about passing this in rather than create it in AM on construction?  If you need to do a close on the ExecutorService, add a close to AM and have the host of the AM class call the close on the way out?  Otherwise, you are exposing the implementation.  If you need configuration setting up the ExecutorService, you have the conf object by doing master.getConfiguration?  There may be other reasons for passing in the ExecutorService that I'm not seeing... Above is just a thought.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4058>

    How do we know this will not result in a double assignment?  What if the .META. is OPENING and the master crashes.   A new master comes up but meantime the .META. has moved to OPENED and here we will assign the new .META. out without checking if .META. successfully OPENED? 
    
    Should we at least check the znode again before going ahead with assign?  Should we actually try grabbing the znode completely before we reassign so if .META. is opening, it will fail to complete?



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4059>

    Thanks for making this little method disableRegionIfInRIT



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4060>

    If you make a new patch, can you fix formatting of this comment?  Its strange the way its broke over a number of lines.
    
    So, the flag here is 'reassign'.  Does that mean 'called from timeout monitor'?  If so, its not clear (Change name of flag or at least fixup javadoc).  Else this comment is hard to understand.  But even so, do we care where its called from?  All thats important is if we are to reassign or not (What if we figure we need to set reassign to true elsewhere than in timeout monitor?  Then this comment would be extra confusing?)



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4062>

    This is great.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4063>

    Should this method be called isDisabledOrDisabling?  Then a reader will expect the boolean return.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4064>

    This javadoc is better in that it says the flag 'reassign' is set if we were invoked by timeout monitor.   Is it important that we're called by the timeout monitor?   Do we need to know about the caller when this method is run?  What if we decide to call this method from elsewhere than from timeout monitor? 



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4065>

    Again is it important that we know we are being called from timeout monitor?  What does it mean when we are called from timeout monitor?  Is it that we are trying to interrupt normal processing because it seems to have failed?  If so, 'reassign' may be the wrong name for this flag?  Would 'hijack' be a better flag name?  Or 'appropriate' or some such?
    
    Sorry for harping on the flag name Ram.  If you have good names that convey intent/purpose, it makes it easier on the reader.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4066>

    Can you fix formatting of this comment?



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4067>

    Should we change znode state here?  Read it, get the znode id and try to change it.  If we fail, then RS still has control?



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4068>

    Making a new method is good here.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4069>

    This is nice.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4070>

    Try not to make these changes going forward (you are moving the + from end of line to start of next line and thats all -- it bulks up your patch unnecessarily).



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4071>

    Nice.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4072>

    Does doing this clear the watch on the znode such that we will not notice if it moves from OPENING to OPENED?



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
<https://reviews.apache.org/r/1668/#comment4073>

    Is there something wrong here?  The comment says do not make znode OFFLINE, wait on timeout monitor, but we are queuing up an ASSIGN it looks like?



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
<https://reviews.apache.org/r/1668/#comment4074>

    Yeah, I wonder if this even belongs out here.  Just have it inside in AM?
    
    Do we even call close on it?  Should we?



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
<https://reviews.apache.org/r/1668/#comment4075>

    No biggie but you could have written this as:
    
    return (versionOfflineNode == -1)? openRegion(region): openRegion(region, versionOffilneNode);



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java
<https://reviews.apache.org/r/1668/#comment4077>

    I'd suggest you move the enum here from AM rather than have this strange import of an inner type.
    
    Maybe you'd have two classes in here.  AssignCallable and UnassignCallable then no need of the enum?
    
    For this class name, do you think we need TimeOutManager in its name?  In fact, is there a TimeoutManager?   It seems to be the ExcecutorService?  Or is it the method invokeTimeoutManager?
    
    Seems like method should be something like invoke and you queue AssignCallable or UnassignCallable?



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java
<https://reviews.apache.org/r/1668/#comment4076>

    misspelling



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
<https://reviews.apache.org/r/1668/#comment4078>

    I think I asked this already and I should check the response above, but is there any duplication here with current openRegion?  Should we factor out common code?



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
<https://reviews.apache.org/r/1668/#comment4079>

    For sure this method should not know about timeout monitor.  This is a class in another package from master AM code.  Should this flag be named 'hijack' or 'appropriate' or 'preempt'



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
<https://reviews.apache.org/r/1668/#comment4082>

    You do not javadoc your return.



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
<https://reviews.apache.org/r/1668/#comment4080>

    Can you clean up the formatting on this comment?  Do we need to know about timeout monitor in here?
    



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
<https://reviews.apache.org/r/1668/#comment4081>

    Should this check be done up in timeout monitor or in a callable so its not inline?  If it still is in OPENING state up in timeout monitor, then come in here?  Just a suggestion (May not be a good one).



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
<https://reviews.apache.org/r/1668/#comment4083>

    Is it correct adding 1?



http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
<https://reviews.apache.org/r/1668/#comment4084>

    This is a strange location for a comment...  I'd say put it into the else clause.


- Michael


On 2011-08-29 12:11:23, ramkrishna vasudevan wrote:
bq.  
bq.  -----------------------------------------------------------
bq.  This is an automatically generated e-mail. To reply, visit:
bq.  https://reviews.apache.org/r/1668/
bq.  -----------------------------------------------------------
bq.  
bq.  (Updated 2011-08-29 12:11:23)
bq.  
bq.  
bq.  Review request for Ted Yu, Michael Stack, Jean-Daniel Cryans, and Jonathan Gray.
bq.  
bq.  
bq.  Summary
bq.  -------
bq.  
bq.  HBASE-4015 - updated patch.
bq.  
bq.  
bq.          invokeTimeOutManager(regionState.getRegion(),
bq.              TimeOutOperationType.ASSIGN);
bq.  
bq.  Instead of storing the state when timeout was deducted we process it
bq.  using future task.
bq.  ================================================================
bq.  
bq.        case RS_ZK_REGION_OPENING:
bq.          // TODO: Could check if it was on deadServers.  If it was, then we could
bq.          // do what happens in TimeoutMonitor when it sees this condition.
bq.  
bq.          // Just insert region into RIT
bq.          // If this never updates the timeout will trigger new assignment
bq.          if (regionInfo.isMetaRegion() || regionInfo.isRootRegion()) {
bq.            regionsInTransition.put(encodedRegionName, new RegionState(
bq.                regionInfo, RegionState.State.OPENING, data.getStamp(), data
bq.                    .getOrigin()));
bq.            processOpeningState(regionInfo);
bq.            break;
bq.          }
bq.          regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
bq.              RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
bq.          break;
bq.  
bq.  This change is for HBASE-4203. META and ROOT table need not wait till timeout
bq.  ======================================================================
bq.  
bq.  In forceRegionStateToOffline()
bq.  
bq.      } else {
bq.        // If invoked from timeout monitor donot force it to OFFLINE. Based on the
bq.        // state we will decide if to change in-memory state to OFFLINE or not.  It will
bq.        // be done before setting the znode to OFFLINE state.
bq.        if (!timeOutMonitorReAllocate) {
bq.          LOG.debug("Forcing OFFLINE; was=" + state);
bq.          state.update(RegionState.State.OFFLINE);
bq.        }
bq.  If the timeout monitor tries to reallcoate the node then dont make
bq.  the inmemory state to OFFLINE.
bq.  But the noraml assign flow doesnot expect the inmemory state to OFFLINE.
bq.  Hence the above change.  This is continued with the check in 
bq.  assign()
bq.  int setOfflineInZooKeeper(final RegionState state,
bq.        boolean timeOutMonitorReAllocate) {
bq.      // If invoked from timeoutmonitor the current state in memory need not be
bq.      // OFFLINE.  
bq.      if (!timeOutMonitorReAllocate && !state.isClosed() && !state.isOffline()) {
bq.            this.master.abort("Unexpected state trying to OFFLINE; " + state,
bq.            new IllegalStateException());
bq.        return -1;
bq.      }
bq.  ======================================================================
bq.  
bq.      boolean allowCreation = false;
bq.      // If the isReAllocate is true and the current state is PENDING_OPEN
bq.      // or OPENING then update the inmemory state to PENDING_OPEN. This is
bq.      // important because
bq.      // if timeoutmonitor deducts that a region was in OPENING state for a long
bq.      // time but by the
bq.      // time timeout monitor tranits the node to OFFLINE the RS would have opened
bq.      // the node and the
bq.      // state in znode will be RS_ZK_REGION_OPENED. Inorder to invoke the
bq.      // OpenedRegionHandler
bq.      // we expect the inmemeory state to be PENDING_OPEN or OPENING.
bq.      // For all other cases we can change the inmemory state to OFFLINE.
bq.      if (timeOutMonitorReAllocate
bq.          && (state.getState().equals(RegionState.State.PENDING_OPEN) || state
bq.              .getState().equals(RegionState.State.OPENING))) {
bq.        state.update(RegionState.State.PENDING_OPEN);
bq.        allowCreation = false;
bq.      } else {
bq.        state.update(RegionState.State.OFFLINE);
bq.        allowCreation = true;
bq.      }
bq.  This change is quite tricky.  
bq.  In normal assign flow the unassigned node for the region will not be present
bq.  Hence we need to allow the creation of the node newly.
bq.  But in timeout monitor case we will have the node present in some state hence 
bq.  we decide whether to create node newly or not inside ZKAssign.createOrForceNodeOffline
bq.  
bq.  The above code also updates the inmemory state of OFFLINE or PENDING_OPEN
bq.  which was not update in the previous forceRegionStateToOffline() call.
bq.  ==============================================================================
bq.  In ZKAssign.java()
bq.      if (version == -1) {
bq.        // If timeoutmonitor deducts a node to be in OPENING state but before it
bq.        // could
bq.        // transit to OFFLINE state if RS had opened the region then the Master
bq.        // deletes the
bq.        // assigned region znode. In that case the znode will not exist. So we
bq.        // should not
bq.        // create the znode again which will lead to double assignment.
bq.        if (timeOutMonitorReAllocate && !allowCreation) {
bq.          return -1;
bq.        }
bq.  this part prevents double assignment
bq.  If timeoutmonitor tries to force to OFFLINE state an existing region which was in RIT
bq.  but before it could do that if the node was opened then openedregionhandler will delete
bq.  the node hence we should not create the node again.
bq.  
bq.  =======================================================================================
bq.  In ZkAssign.java()
bq.      } else {
bq.        RegionTransitionData curDataInZNode = ZKAssign.getDataNoWatch(zkw, region
bq.            .getEncodedName(), stat);
bq.        // Do not move the node to OFFLINE if znode is in any of the following
bq.        // state.
bq.        // Because these are already executed states.
bq.        if (timeOutMonitorReAllocate && null != curDataInZNode) {
bq.          EventType eventType = curDataInZNode.getEventType();
bq.          if (eventType.equals(EventType.RS_ZK_REGION_CLOSING)
bq.              || eventType.equals(EventType.RS_ZK_REGION_CLOSED)
bq.              || eventType.equals(EventType.RS_ZK_REGION_OPENED)) {
bq.            return -1;
bq.          }
bq.        }
bq.  This check prevents from moving the node to OFFLINE state if just before
bq.  the node is tried to force to OFFLINE the RS would have changed the state 
bq.  to either CLOSING or CLOSED or OPENED.  now again moving to OFFLINE
bq.  will lead to douoble assignment and will an additional operaition.
bq.  
bq.  ====================================================================================
bq.  In ZKassign.java()
bq.        boolean setData = false;
bq.        try {
bq.          setData = ZKUtil.setData(zkw, node, data.getBytes(), version);
bq.          // Setdata throws KeeperException which aborts the Master. So we are
bq.          // catching it here.
bq.          // If just before setting the znode to OFFLINE if the RS has made any
bq.          // change to the
bq.          // znode state then we need to return -1.
bq.        } catch (KeeperException kpe) {
bq.          LOG.info("Version mismatch while setting the node to OFFLINE state.");
bq.          return -1;
bq.        }
bq.  This change is actually to avoid the master from abortng. If the forceful OFFLINE is in
bq.  progrss but just before setting the RS has changed the state.
bq.  Then the setData will fail leading to master abort.
bq.  Hence we are catching the exception.
bq.  ====================================================================================
bq.  
bq.  In assignmentManager.java
bq.  +      if (setOfflineInZK && versionOfOfflineNode == -1)
bq.  +        return;
bq.  This is nothing but the refactoring done in the existing code.
bq.  -     if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
bq.  So if setting the version is unsuccessful return.
bq.  =====================================================================================
bq.  In ZKassign.java()
bq.  // the below check ensure that double assignment doesnot happen.
bq.      // When the node is created for the first time then the expected version
bq.      // that is
bq.      // passed will be -1 and the version in znode will be 0.
bq.      // In all other cases the version in znode will be > 0.
bq.      else if (beginState.equals(EventType.M_ZK_REGION_OFFLINE)
bq.          && endState.equals(EventType.RS_ZK_REGION_OPENING)
bq.          && expectedVersion == -1 && stat.getVersion() != 0) {
bq.        LOG.warn(zkw.prefix("Attempt to transition the " + "unassigned node for "
bq.            + encoded + " from " + beginState + " to " + endState + " failed, "
bq.            + "the node existed but was version " + stat.getVersion()
bq.            + " not the expected version " + expectedVersion));
bq.        return -1;
bq.      }
bq.  As the comment explains when the node is created for first time the expectedversion will
bq.  be -1 but the actual version will be 0.  Here the scenario is
bq.  If RS1 has not tranitioned the node from OFFLINE to OPENING if
bq.  RS2 gets the call from Master after forcefully chaning to OFFLINE the
bq.  Rs2 will take the control of the node.
bq.  At that time if RS1 starts transmitting the node then we should not allow
bq.  it.
bq.  ===============================================================================
bq.  
bq.  
bq.  This addresses bug HBASE-4015.
bq.      https://issues.apache.org/jira/browse/HBASE-4015
bq.  
bq.  
bq.  Diffs
bq.  -----
bq.  
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/ipc/HRegionInterface.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/TimeOutManagerCallable.java PRE-CREATION 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenMetaHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java 1161985 
bq.    http://svn.apache.org/repos/asf/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java 1161985 
bq.  
bq.  Diff: https://reviews.apache.org/r/1668/diff
bq.  
bq.  
bq.  Testing
bq.  -------
bq.  
bq.  Yes.  But could not add new test case.
bq.  TestMasterFailOver is passing with the current changes also.
bq.  
bq.  
bq.  Thanks,
bq.  
bq.  ramkrishna
bq.  
bq.



> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, HBASE-4015_2_trunk.patch, HBASE-4015_reprepared_trunk_2.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-4015) Refactor the TimeoutMonitor to make it less racy

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-4015?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

ramkrishna.s.vasudevan updated HBASE-4015:
------------------------------------------

    Attachment: HBASE-4015_1_trunk.patch

> Refactor the TimeoutMonitor to make it less racy
> ------------------------------------------------
>
>                 Key: HBASE-4015
>                 URL: https://issues.apache.org/jira/browse/HBASE-4015
>             Project: HBase
>          Issue Type: Sub-task
>    Affects Versions: 0.90.3
>            Reporter: Jean-Daniel Cryans
>            Assignee: ramkrishna.s.vasudevan
>            Priority: Blocker
>             Fix For: 0.92.0
>
>         Attachments: HBASE-4015_1_trunk.patch, Timeoutmonitor with state diagrams.pdf
>
>
> The current implementation of the TimeoutMonitor acts like a race condition generator, mostly making things worse rather than better. It does it's own thing for a while without caring for what's happening in the rest of the master.
> The first thing that needs to happen is that the regions should not be processed in one big batch, because that sometimes can take minutes to process (meanwhile a region that timed out opening might have opened, then what happens is it will be reassigned by the TimeoutMonitor generating the never ending PENDING_OPEN situation).
> Those operations should also be done more atomically, although I'm not sure how to do it in a scalable way in this case.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira