HBase
  1. HBase
  2. HBASE-7799

reassigning region stuck in open still may not work correctly due to leftover ZK node

    Details

    • Type: Bug Bug
    • Status: Closed
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 0.95.0
    • Component/s: None
    • Labels:
      None
    • Hadoop Flags:
      Reviewed

      Description

      (logs grepped by region name, and abridged.

      META server was dead so OpenRegionHandler for the region took a while, and was interrupted:

      2013-02-08 14:35:01,555 DEBUG [RS_OPEN_REGION-10.11.2.92,64485,1360362800564-2] handler.OpenRegionHandler(255): Interrupting thread Thread[PostOpenDeployTasks:871d1c3bdf98a2c93b527cb6cc61327d,5,main]
      

      Then master tried to force region offline and reassign:

      2013-02-08 14:35:06,500 INFO  [MASTER_SERVER_OPERATIONS-10.11.2.92,64483,1360362800340-1] master.RegionStates(347): Found opening region {IntegrationTestRebalanceAndKillServersTargeted,7333332c,1360362805563.871d1c3bdf98a2c93b527cb6cc61327d. state=OPENING, ts=1360362901596, server=10.11.2.92,64485,1360362800564} to be reassigned by SSH for 10.11.2.92,64485,1360362800564
      2013-02-08 14:35:06,500 INFO  [MASTER_SERVER_OPERATIONS-10.11.2.92,64483,1360362800340-1] master.RegionStates(242): Region {NAME => 'IntegrationTestRebalanceAndKillServersTargeted,7333332c,1360362805563.871d1c3bdf98a2c93b527cb6cc61327d.', STARTKEY => '7333332c', ENDKEY => '7ffffff8', ENCODED => 871d1c3bdf98a2c93b527cb6cc61327d,} transitioned from {IntegrationTestRebalanceAndKillServersTargeted,7333332c,1360362805563.871d1c3bdf98a2c93b527cb6cc61327d. state=OPENING, ts=1360362901596, server=10.11.2.92,64485,1360362800564} to {IntegrationTestRebalanceAndKillServersTargeted,7333332c,1360362805563.871d1c3bdf98a2c93b527cb6cc61327d. state=CLOSED, ts=1360362906500, server=null}
      2013-02-08 14:35:06,505 DEBUG [10.11.2.92,64483,1360362800340-GeneralBulkAssigner-1] master.AssignmentManager(1530): Forcing OFFLINE; was={IntegrationTestRebalanceAndKillServersTargeted,7333332c,1360362805563.871d1c3bdf98a2c93b527cb6cc61327d. state=CLOSED, ts=1360362906500, server=null}
      2013-02-08 14:35:06,506 DEBUG [10.11.2.92,64483,1360362800340-GeneralBulkAssigner-1] zookeeper.ZKAssign(176): master:64483-0x13cbbf1025d0000 Async create of unassigned node for 871d1c3bdf98a2c93b527cb6cc61327d with OFFLINE state
      

      But didn't delete the original ZK node?

      2013-02-08 14:35:06,509 WARN  [main-EventThread] master.OfflineCallback(59): Node for /hbase/region-in-transition/871d1c3bdf98a2c93b527cb6cc61327d already exists
      2013-02-08 14:35:06,509 DEBUG [main-EventThread] master.OfflineCallback(69): rs={IntegrationTestRebalanceAndKillServersTargeted,7333332c,1360362805563.871d1c3bdf98a2c93b527cb6cc61327d. state=OFFLINE, ts=1360362906506, server=null}, server=10.11.2.92,64488,1360362800651
      2013-02-08 14:35:06,512 DEBUG [main-EventThread] master.OfflineCallback$ExistCallback(106): rs={IntegrationTestRebalanceAndKillServersTargeted,7333332c,1360362805563.871d1c3bdf98a2c93b527cb6cc61327d. state=OFFLINE, ts=1360362906506, server=null}, server=10.11.2.92,64488,1360362800651
      

      So it went into infinite cycle of failing to assign due to this:

      2013-02-08 14:35:06,517 INFO  [PRI IPC Server handler 7 on 64488] regionserver.HRegionServer(3435): Received request to open region: IntegrationTestRebalanceAndKillServersTargeted,7333332c,1360362805563.871d1c3bdf98a2c93b527cb6cc61327d. on 10.11.2.92,64488,1360362800651
      2013-02-08 14:35:06,521 WARN  [RS_OPEN_REGION-10.11.2.92,64488,1360362800651-0] zookeeper.ZKAssign(762): regionserver:64488-0x13cbbf1025d0004 Attempt to transition the unassigned node for 871d1c3bdf98a2c93b527cb6cc61327d from M_ZK_REGION_OFFLINE to RS_ZK_REGION_OPENING failed, the node existed but was in the state RS_ZK_REGION_OPENING set by the server [wrong server name redacted, see HBASE-7798]
      

      Transitioning failed-to-open similarly fails.

      It seems like master needs to nuke ZK node unconditionally to offline?

        Issue Links

          Activity

          Hide
          Sergey Shelukhin added a comment -

          I will look at this next week by default?

          Show
          Sergey Shelukhin added a comment - I will look at this next week by default?
          Hide
          ramkrishna.s.vasudevan added a comment -

          Forecful OFFLINE is to change it forcefully irrespective of the previou state right?
          I may be wrong here.

          Show
          ramkrishna.s.vasudevan added a comment - Forecful OFFLINE is to change it forcefully irrespective of the previou state right? I may be wrong here.
          Hide
          Sergey Shelukhin added a comment -

          I'd think so, but it looks like it didn't work, master says that node already exists and doesn't indicate that it's overwritten.

          Show
          Sergey Shelukhin added a comment - I'd think so, but it looks like it didn't work, master says that node already exists and doesn't indicate that it's overwritten.
          Hide
          ramkrishna.s.vasudevan added a comment -

          Okie. If you can attach the logs it would be great. I can take a more closer look at this if you don't mind.

          Show
          ramkrishna.s.vasudevan added a comment - Okie. If you can attach the logs it would be great. I can take a more closer look at this if you don't mind.
          Hide
          Sergey Shelukhin added a comment -

          Attaching log. Note that it has a buggy experimental feature which currently makes HCM retry longer, but this should have no bearing on the problem...

          Show
          Sergey Shelukhin added a comment - Attaching log. Note that it has a buggy experimental feature which currently makes HCM retry longer, but this should have no bearing on the problem...
          Hide
          ramkrishna.s.vasudevan added a comment -

          Found the problem.
          Straight forward scenario.
          A region is in transition and getting moved to RS1. RS1 gets killed in between and the state in zk is RS_ZK_OPENING.
          Now SSH kicks in. Jimmy's HBAS-7701 intelligently picks this region so that SSH can start assigning it.

          The RIT state is forcefully made to CLOSE. And then the GeneralBulkAssigner kicks in.
          Now here try to create a znode with OFFLINE state. But if node exists we silently return.
          In AM.asyncSetOfflineInZooKeeper()

          try {
                ZKAssign.asyncCreateNodeOffline(watcher, state.getRegion(),
                  destination, cb, state);
              } catch (KeeperException e) {
                if (e instanceof NodeExistsException) {
                  LOG.warn("Node for " + state.getRegion() + " already exists");
                } else {
                  server.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
                }
                return false;
          

          Now when the new RS2 tries to transition the znode thinking it to be in M_ZK_OFFLINE state it does not happen. Thus leading to infinite loop.
          Patch i will come up later as its late here.
          Pls correct me if am wrong here.

          Show
          ramkrishna.s.vasudevan added a comment - Found the problem. Straight forward scenario. A region is in transition and getting moved to RS1. RS1 gets killed in between and the state in zk is RS_ZK_OPENING. Now SSH kicks in. Jimmy's HBAS-7701 intelligently picks this region so that SSH can start assigning it. The RIT state is forcefully made to CLOSE. And then the GeneralBulkAssigner kicks in. Now here try to create a znode with OFFLINE state. But if node exists we silently return. In AM.asyncSetOfflineInZooKeeper() try { ZKAssign.asyncCreateNodeOffline(watcher, state.getRegion(), destination, cb, state); } catch (KeeperException e) { if (e instanceof NodeExistsException) { LOG.warn( "Node for " + state.getRegion() + " already exists" ); } else { server.abort( "Unexpected ZK exception creating/setting node OFFLINE" , e); } return false ; Now when the new RS2 tries to transition the znode thinking it to be in M_ZK_OFFLINE state it does not happen. Thus leading to infinite loop. Patch i will come up later as its late here. Pls correct me if am wrong here.
          Hide
          ramkrishna.s.vasudevan added a comment -

          One interesting thing in AM is every time we get new type of issues .

          Show
          ramkrishna.s.vasudevan added a comment - One interesting thing in AM is every time we get new type of issues .
          Hide
          Sergey Shelukhin added a comment -

          Yeah, I was starting to come to a similar conclusion yesterday, although I didn't finish looking (that "forcing offline" is "forcing" only internal state, not ZK).
          Wrt issues - there was HBASE-5something-or-other JIRA about making assignment simpler by getting rid/replacing/etc. things

          Show
          Sergey Shelukhin added a comment - Yeah, I was starting to come to a similar conclusion yesterday, although I didn't finish looking (that "forcing offline" is "forcing" only internal state, not ZK). Wrt issues - there was HBASE-5 something-or-other JIRA about making assignment simpler by getting rid/replacing/etc. things
          Hide
          ramkrishna.s.vasudevan added a comment -

          One correction here. I think the code in AM.asyncSetOfflineInZooKeeper() is fine. But the OfflineCallBack just does not throw NodeExistsException in processResult.

           @Override
            public void processResult(int rc, String path, Object ctx, String name) {
              if (rc == KeeperException.Code.NODEEXISTS.intValue()) {
                LOG.warn("Node for " + path + " already exists");
              }
          

          I see the above log in the log file and not the "LOG.warn("Node for " + state.getRegion() + " already exists");" in AM.

          master.OfflineCallback(59): Node for /hbase/region-in-transition/871d1c3bdf98a2c93b527cb6cc61327d already exists
          

          So the OfflineCallBack is eating up the NodeExistsException. I may be wrong here because i am new to this CallBack code introduced in trunk.

          Show
          ramkrishna.s.vasudevan added a comment - One correction here. I think the code in AM.asyncSetOfflineInZooKeeper() is fine. But the OfflineCallBack just does not throw NodeExistsException in processResult. @Override public void processResult( int rc, String path, Object ctx, String name) { if (rc == KeeperException.Code.NODEEXISTS.intValue()) { LOG.warn( "Node for " + path + " already exists" ); } I see the above log in the log file and not the "LOG.warn("Node for " + state.getRegion() + " already exists");" in AM. master.OfflineCallback(59): Node for /hbase/region-in-transition/871d1c3bdf98a2c93b527cb6cc61327d already exists So the OfflineCallBack is eating up the NodeExistsException. I may be wrong here because i am new to this CallBack code introduced in trunk.
          Hide
          ramkrishna.s.vasudevan added a comment -

          Debugging the code and reproduced the problem.
          What i observed was

          try {
                ZKAssign.asyncCreateNodeOffline(watcher, state.getRegion(),
                  destination, cb, state);
              } catch (KeeperException e) {
                if (e instanceof NodeExistsException) {
                  LOG.warn("Node for " + state.getRegion() + " already exists");
                } else {
                  server.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
                }
                return false;
              }
          return true;
          

          The asyncCreateNodeOffline just always returns true because it does not wait for the callback to take action. Also the callback does not throw NodeExistsException.
          In short the catch block is a dead code.
          So should we make this synchronous or wait till the callback processes the current zk event? The same thing exists in 0.94 also but bulk assign is not used in SSH except for Create table and enable table.

          Show
          ramkrishna.s.vasudevan added a comment - Debugging the code and reproduced the problem. What i observed was try { ZKAssign.asyncCreateNodeOffline(watcher, state.getRegion(), destination, cb, state); } catch (KeeperException e) { if (e instanceof NodeExistsException) { LOG.warn( "Node for " + state.getRegion() + " already exists" ); } else { server.abort( "Unexpected ZK exception creating/setting node OFFLINE" , e); } return false ; } return true ; The asyncCreateNodeOffline just always returns true because it does not wait for the callback to take action. Also the callback does not throw NodeExistsException. In short the catch block is a dead code. So should we make this synchronous or wait till the callback processes the current zk event? The same thing exists in 0.94 also but bulk assign is not used in SSH except for Create table and enable table.
          Hide
          Jimmy Xiang added a comment - - edited

          So should we make this synchronous or wait till the callback processes the current zk event?

          I think we still need to do async for performance reason. This issue should be fixed somewhere else.

          Good catch. @Ram, I think it is because I didn't delete the assign znode. Can you try this patch? Thanks.

          Show
          Jimmy Xiang added a comment - - edited So should we make this synchronous or wait till the callback processes the current zk event? I think we still need to do async for performance reason. This issue should be fixed somewhere else. Good catch. @Ram, I think it is because I didn't delete the assign znode. Can you try this patch? Thanks.
          Hide
          Sergey Shelukhin added a comment -

          On patch - is it final, or just preliminary?

          Should there be separate JIRA for fixing the issue that Ramkrishna indicates?

          Show
          Sergey Shelukhin added a comment - On patch - is it final, or just preliminary? Should there be separate JIRA for fixing the issue that Ramkrishna indicates?
          Hide
          ramkrishna.s.vasudevan added a comment -

          Jimmy Xiang
          The deletion is needed. But what am thinking is i remember we made this change when Chunhui made SSH as bulk assignment rather than individual assignment.
          May be we missed out that code while making the new changes.

          Should there be separate JIRA for fixing the issue that Ramkrishna indicates?

          May be its needed for sure.

          @Sergey
          Does your integration test suite run with this? I will also try doing some manual test.

          Show
          ramkrishna.s.vasudevan added a comment - Jimmy Xiang The deletion is needed. But what am thinking is i remember we made this change when Chunhui made SSH as bulk assignment rather than individual assignment. May be we missed out that code while making the new changes. Should there be separate JIRA for fixing the issue that Ramkrishna indicates? May be its needed for sure. @Sergey Does your integration test suite run with this? I will also try doing some manual test.
          Hide
          ramkrishna.s.vasudevan added a comment -

          Anyway let me check more on this later today.

          Show
          ramkrishna.s.vasudevan added a comment - Anyway let me check more on this later today.
          Hide
          ramkrishna.s.vasudevan added a comment -

          This patch solves the problem.
          Going thro that area of the code i see one more issue,
          If suppose SSH is going on for an RS where one of the region is yet to open. The META update has not happened.
          The Am.procesServerShutdown() picks up this region as it is in RIT.
          But the hris obtained by META scan does not have it in SSH. So i dont find this region getting populated in toAssignRegions.
          The same has been done in the recent checkin in 0.94 (HBASE-7521).
          Correct me if am wrong? We can raise another JIRA and solve it. Also the callback issue that i mentioned can be solved in another JIRA.

          Show
          ramkrishna.s.vasudevan added a comment - This patch solves the problem. Going thro that area of the code i see one more issue, If suppose SSH is going on for an RS where one of the region is yet to open. The META update has not happened. The Am.procesServerShutdown() picks up this region as it is in RIT. But the hris obtained by META scan does not have it in SSH. So i dont find this region getting populated in toAssignRegions. The same has been done in the recent checkin in 0.94 ( HBASE-7521 ). Correct me if am wrong? We can raise another JIRA and solve it. Also the callback issue that i mentioned can be solved in another JIRA.
          Hide
          Jimmy Xiang added a comment -

          @Ram, thanks a lot for verifying the patch.

          But the hris obtained by META scan does not have it in SSH. So i dont find this region getting populated in toAssignRegions

          If Am.procesServerShutdown() picks it up, then if should be in toAssignRegions since we add them all even it is not in hris:

           SSH
                List<HRegionInfo> toAssignRegions = new ArrayList<HRegionInfo>();
                toAssignRegions.addAll(regionsInTransition);
          

          Am.processServerShutdown could skip a region if it is not pending open any more:

           AM
                  if (regionState == null
                      || !regionState.isPendingOpenOrOpeningOnServer(sn)) {
                    LOG.info("Skip region " + hri
                      + " since it is not opening on the dead server any more: " + sn);
                    it.remove();
          

          Could you please check if the region is still pending open?

          As to the callback issue, yes, it can be solved in another jira.

          Show
          Jimmy Xiang added a comment - @Ram, thanks a lot for verifying the patch. But the hris obtained by META scan does not have it in SSH. So i dont find this region getting populated in toAssignRegions If Am.procesServerShutdown() picks it up, then if should be in toAssignRegions since we add them all even it is not in hris: SSH List<HRegionInfo> toAssignRegions = new ArrayList<HRegionInfo>(); toAssignRegions.addAll(regionsInTransition); Am.processServerShutdown could skip a region if it is not pending open any more: AM if (regionState == null || !regionState.isPendingOpenOrOpeningOnServer(sn)) { LOG.info( "Skip region " + hri + " since it is not opening on the dead server any more: " + sn); it.remove(); Could you please check if the region is still pending open? As to the callback issue, yes, it can be solved in another jira.
          Hide
          ramkrishna.s.vasudevan added a comment -

          Am sorry Jimmy. Don know how i missed that line. I did some changes there for testing something so i deleted that line

          Show
          ramkrishna.s.vasudevan added a comment - Am sorry Jimmy. Don know how i missed that line. I did some changes there for testing something so i deleted that line
          Hide
          Jimmy Xiang added a comment -

          No problem. Should I commit this patch?

          Show
          Jimmy Xiang added a comment - No problem. Should I commit this patch?
          Hide
          ramkrishna.s.vasudevan added a comment -

          Ok..I am +1 on patch. May be Sergey wants to take a look?

          Show
          ramkrishna.s.vasudevan added a comment - Ok..I am +1 on patch. May be Sergey wants to take a look?
          Hide
          Jimmy Xiang added a comment -

          Sure. Sergey Shelukhin, any comments?

          Show
          Jimmy Xiang added a comment - Sure. Sergey Shelukhin , any comments?
          Hide
          Sergey Shelukhin added a comment -

          looks good to me... I will file jira to fix the other part

          Show
          Sergey Shelukhin added a comment - looks good to me... I will file jira to fix the other part
          Hide
          Jimmy Xiang added a comment -

          Integrated into trunk. Thanks Ram a lot for reviewing and verifying the patch. Thanks Sergey a lot for reviewing the patch and reporting the issue.

          Show
          Jimmy Xiang added a comment - Integrated into trunk. Thanks Ram a lot for reviewing and verifying the patch. Thanks Sergey a lot for reviewing the patch and reporting the issue.
          Hide
          Hudson added a comment -

          Integrated in HBase-TRUNK #3881 (See https://builds.apache.org/job/HBase-TRUNK/3881/)
          HBASE-7799 reassigning region stuck in open still may not work correctly due to leftover ZK node (Revision 1446784)

          Result = FAILURE
          jxiang :
          Files :

          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
          Show
          Hudson added a comment - Integrated in HBase-TRUNK #3881 (See https://builds.apache.org/job/HBase-TRUNK/3881/ ) HBASE-7799 reassigning region stuck in open still may not work correctly due to leftover ZK node (Revision 1446784) Result = FAILURE jxiang : Files : /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
          Hide
          Hudson added a comment -

          Integrated in HBase-TRUNK-on-Hadoop-2.0.0 #409 (See https://builds.apache.org/job/HBase-TRUNK-on-Hadoop-2.0.0/409/)
          HBASE-7799 reassigning region stuck in open still may not work correctly due to leftover ZK node (Revision 1446784)

          Result = FAILURE
          jxiang :
          Files :

          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
          Show
          Hudson added a comment - Integrated in HBase-TRUNK-on-Hadoop-2.0.0 #409 (See https://builds.apache.org/job/HBase-TRUNK-on-Hadoop-2.0.0/409/ ) HBASE-7799 reassigning region stuck in open still may not work correctly due to leftover ZK node (Revision 1446784) Result = FAILURE jxiang : Files : /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
          Hide
          stack added a comment -

          Marking closed.

          Show
          stack added a comment - Marking closed.

            People

            • Assignee:
              Jimmy Xiang
              Reporter:
              Sergey Shelukhin
            • Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development