hbase-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Allan Yang (JIRA)" <j...@apache.org>
Subject [jira] [Created] (HBASE-17264) Process RIT with offline state will always fail to open in the first time
Date Tue, 06 Dec 2016 11:11:27 GMT
Allan Yang created HBASE-17264:

             Summary: Process RIT with offline state will always fail to open in the first
                 Key: HBASE-17264
                 URL: https://issues.apache.org/jira/browse/HBASE-17264
             Project: HBase
          Issue Type: Bug
          Components: Region Assignment
    Affects Versions: 1.1.7
            Reporter: Allan Yang
            Assignee: Allan Yang
         Attachments: HBASE-17264-branch1.1.patch

In Assignment#processRegionsInTransition, when handling regions with M_ZK_REGION_OFFLINE state,
we used a handler to reassign this region. But, when calling assign, we passed not to set
the zk node 
        // Insert in RIT and resend to the regionserver
        regionStates.updateRegionState(rt, State.PENDING_OPEN);
        final RegionState rsOffline = regionStates.getRegionState(regionInfo);
          new EventHandler(server, EventType.M_MASTER_RECOVERY) {
            public void process() throws IOException {
              ReentrantLock lock = locker.acquireLock(regionInfo.getEncodedName());
              try {
                RegionPlan plan = new RegionPlan(regionInfo, null, sn);
                addPlan(encodedName, plan);
                assign(rsOffline, false, false);  //we decide to not to setOfflineInZK
              } finally {
But, when setOfflineInZK is false, we passed a zk node vesion of -1 to the regionserver, meaning
the zk node does not exists. But actually the offline zk node does exist with a different
version. RegionServer will report fail to open because of this.
This situation is trully happened in our test environment. Though the master will recevied
the FAILED_OPEN zk event and retry later, but due to a another bug(I will open another jira
later). The Region will be remain in closed state forever.

Master assign region in RIT
2016-11-23 17:11:46,842 INFO  [example.org:30001.activeMasterManager] master.AssignmentManager:
Processing 57513956a7b671f4e8da1598c2e2970e in state: M_ZK_REGION_OFFLINE
2016-11-23 17:11:46,842 INFO  [example.org:30001.activeMasterManager] master.RegionStates:
Transition {57513956a7b671f4e8da1598c2e2970e state=OFFLINE, ts=1479892306738, server=example.org,30003,1475893095003}
to {57513956a7b671f4e8da1598c2e2970e state=PENDING_OPEN, ts=1479892306842, server=example.org,30003,1479780976834}
2016-11-23 17:11:46,842 INFO  [example.org:30001.activeMasterManager] master.AssignmentManager:
Processed region 57513956a7b671f4e8da1598c2e2970e in state M_ZK_REGION_OFFLINE, on server:
2016-11-23 17:11:46,843 INFO  [MASTER_SERVER_OPERATIONS-example.org:30001-0] master.AssignmentManager:
Assigning test,QFO7M,1475986053104.57513956a7b671f4e8da1598c2e2970e. to example.org,30003,1479780976834

RegionServer recevied the open region request, and new a RegionOpenHandler to open the region,
but only to find the RIT node's version is not as it expected. RS transition the RIT ZK node
to failed open in the end
2016-11-23 17:11:46,860 WARN  [RS_OPEN_REGION-example.org:30003-1] coordination.ZkOpenRegionCoordination:
Failed transition from OFFLINE to OPENING for region=57513956a7b671f4e8da1598c2e2970e
2016-11-23 17:11:46,861 WARN  [RS_OPEN_REGION-example.org:30003-1] handler.OpenRegionHandler:
Region was hijacked? Opening cancelled for encodedName=57513956a7b671f4e8da1598c2e2970e
2016-11-23 17:11:46,860 WARN  [RS_OPEN_REGION-example.org:30003-1] zookeeper.ZKAssign: regionserver:30003-0x15810b5f633015f,
quorum=hbase4dev04.et2sqa:2181,hbase4dev05.et2sqa:2181,hbase4dev06.et2sqa:2181, baseZNode=/test-hbase11-func2
Attempt to transition the unassigned node for 57513956a7b671f4e8da1598c2e2970e from M_ZK_REGION_OFFLINE
to RS_ZK_REGION_OPENING failed, the node existed but was version 3 not the expected version

Master recevied this zk event and begin to handle RS_ZK_REGION_FAILED_OPEN
2016-11-23 17:11:46,944 DEBUG [AM.ZK.Worker-pool2-t1] master.AssignmentManager: Handling RS_ZK_REGION_FAILED_OPEN,
server=example.org,30003,1479780976834, region=57513956a7b671f4e8da1598c2e2970e, current_state={57513956a7b671f4e8da1598c2e2970e
state=PENDING_OPEN, ts=1479892306843, server=example.org,30003,1479780976834}

This message was sent by Atlassian JIRA

View raw message