1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.master;
19  
20  import static org.junit.Assert.assertEquals;
21  import static org.junit.Assert.assertFalse;
22  import static org.junit.Assert.assertNotSame;
23  import static org.junit.Assert.assertTrue;
24  import static org.junit.Assert.fail;
25  
26  import java.io.IOException;
27  import java.util.ArrayList;
28  import java.util.HashMap;
29  import java.util.List;
30  import java.util.Map;
31  import java.util.concurrent.atomic.AtomicBoolean;
32  
33  import org.apache.hadoop.hbase.HBaseConfiguration;
34  import org.apache.hadoop.hbase.HBaseTestingUtility;
35  import org.apache.hadoop.hbase.HConstants;
36  import org.apache.hadoop.hbase.HRegionInfo;
37  import org.apache.hadoop.hbase.HServerLoad;
38  import org.apache.hadoop.hbase.KeyValue;
39  import org.apache.hadoop.hbase.MediumTests;
40  import org.apache.hadoop.hbase.Server;
41  import org.apache.hadoop.hbase.ServerName;
42  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
43  import org.apache.hadoop.hbase.catalog.CatalogTracker;
44  import org.apache.hadoop.hbase.client.Get;
45  import org.apache.hadoop.hbase.client.HConnection;
46  import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
47  import org.apache.hadoop.hbase.client.Result;
48  import org.apache.hadoop.hbase.client.Scan;
49  import org.apache.hadoop.hbase.executor.EventHandler.EventType;
50  import org.apache.hadoop.hbase.executor.ExecutorService;
51  import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
52  import org.apache.hadoop.hbase.executor.RegionTransitionData;
53  import org.apache.hadoop.hbase.ipc.HRegionInterface;
54  import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
55  import org.apache.hadoop.hbase.master.AssignmentManager.RegionState.State;
56  import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
57  import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
58  import org.apache.hadoop.hbase.util.Bytes;
59  import org.apache.hadoop.hbase.util.Pair;
60  import org.apache.hadoop.hbase.util.Threads;
61  import org.apache.hadoop.hbase.util.Writables;
62  import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
63  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
64  import org.apache.hadoop.hbase.zookeeper.ZKTable.TableState;
65  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
66  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
67  import org.apache.zookeeper.KeeperException;
68  import org.apache.zookeeper.KeeperException.NodeExistsException;
69  import org.apache.zookeeper.Watcher;
70  import org.junit.After;
71  import org.junit.AfterClass;
72  import org.junit.Before;
73  import org.junit.BeforeClass;
74  import org.junit.Test;
75  import org.junit.experimental.categories.Category;
76  import org.mockito.Mockito;
77  import org.mockito.internal.util.reflection.Whitebox;
78  
79  import com.google.protobuf.ServiceException;
80  
81  
82  /**
83   * Test {@link AssignmentManager}
84   */
85  @Category(MediumTests.class)
86  public class TestAssignmentManager {
87    private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
88    private static final ServerName SERVERNAME_A =
89      new ServerName("example.org", 1234, 5678);
90    private static final ServerName SERVERNAME_B =
91      new ServerName("example.org", 0, 5678);
92    private static final HRegionInfo REGIONINFO =
93      new HRegionInfo(Bytes.toBytes("t"),
94        HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
95    private static final HRegionInfo REGIONINFO_2 = new HRegionInfo(Bytes.toBytes("t"),
96        Bytes.toBytes("a"),Bytes.toBytes( "b"));
97    private static int assignmentCount;
98    private static boolean enabling = false;  
99  
100   // Mocked objects or; get redone for each test.
101   private Server server;
102   private ServerManager serverManager;
103   private ZooKeeperWatcher watcher;
104   private LoadBalancer balancer;
105 
106   @BeforeClass
107   public static void beforeClass() throws Exception {
108     HTU.startMiniZKCluster();
109   }
110 
111   @AfterClass
112   public static void afterClass() throws IOException {
113     HTU.shutdownMiniZKCluster();
114   }
115 
116   @Before
117   public void before() throws ZooKeeperConnectionException, IOException {
118     // TODO: Make generic versions of what we do below and put up in a mocking
119     // utility class or move up into HBaseTestingUtility.
120 
121     // Mock a Server.  Have it return a legit Configuration and ZooKeeperWatcher.
122     // If abort is called, be sure to fail the test (don't just swallow it
123     // silently as is mockito default).
124     this.server = Mockito.mock(Server.class);
125     Mockito.when(server.getConfiguration()).thenReturn(HTU.getConfiguration());
126     this.watcher =
127       new ZooKeeperWatcher(HTU.getConfiguration(), "mockedServer", this.server, true);
128     Mockito.when(server.getZooKeeper()).thenReturn(this.watcher);
129     Mockito.doThrow(new RuntimeException("Aborted")).
130       when(server).abort(Mockito.anyString(), (Throwable)Mockito.anyObject());
131 
132     // Mock a ServerManager.  Say server SERVERNAME_{A,B} are online.  Also
133     // make it so if close or open, we return 'success'.
134     this.serverManager = Mockito.mock(ServerManager.class);
135     Mockito.when(this.serverManager.isServerOnline(SERVERNAME_A)).thenReturn(true);
136     Mockito.when(this.serverManager.isServerOnline(SERVERNAME_B)).thenReturn(true);
137     final Map<ServerName, HServerLoad> onlineServers = new HashMap<ServerName, HServerLoad>();
138     onlineServers.put(SERVERNAME_B, new HServerLoad());
139     onlineServers.put(SERVERNAME_A, new HServerLoad());
140     Mockito.when(this.serverManager.getOnlineServersList()).thenReturn(
141         new ArrayList<ServerName>(onlineServers.keySet()));
142     Mockito.when(this.serverManager.getOnlineServers()).thenReturn(onlineServers);
143     Mockito.when(this.serverManager.sendRegionClose(SERVERNAME_A, REGIONINFO, -1)).
144       thenReturn(true);
145     Mockito.when(this.serverManager.sendRegionClose(SERVERNAME_B, REGIONINFO, -1)).
146       thenReturn(true);
147     // Ditto on open.
148     Mockito.when(this.serverManager.sendRegionOpen(SERVERNAME_A, REGIONINFO, -1)).
149       thenReturn(RegionOpeningState.OPENED);
150     Mockito.when(this.serverManager.sendRegionOpen(SERVERNAME_B, REGIONINFO, -1)).
151     thenReturn(RegionOpeningState.OPENED);
152   }
153 
154   @After
155     public void after() throws KeeperException {
156     if (this.watcher != null) {
157       // Clean up all znodes
158       ZKAssign.deleteAllNodes(this.watcher);
159       this.watcher.close();
160     }
161   }
162 
163   /**
164    * Test a balance going on at same time as a master failover
165    *
166    * @throws IOException
167    * @throws KeeperException
168    * @throws InterruptedException
169    */
170   @Test(timeout = 5000)
171   public void testBalanceOnMasterFailoverScenarioWithOpenedNode()
172       throws IOException, KeeperException, InterruptedException {
173     AssignmentManagerWithExtrasForTesting am =
174       setUpMockedAssignmentManager(this.server, this.serverManager);
175     try {
176       createRegionPlanAndBalance(am, SERVERNAME_A, SERVERNAME_B, REGIONINFO);
177       startFakeFailedOverMasterAssignmentManager(am, this.watcher);
178       while (!am.processRITInvoked) Thread.sleep(1);
179       // Now fake the region closing successfully over on the regionserver; the
180       // regionserver will have set the region in CLOSED state. This will
181       // trigger callback into AM. The below zk close call is from the RS close
182       // region handler duplicated here because its down deep in a private
183       // method hard to expose.
184       int versionid =
185         ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
186       assertNotSame(versionid, -1);
187       Mocking.waitForRegionOfflineInRIT(am, REGIONINFO.getEncodedName());
188 
189       // Get the OFFLINE version id.  May have to wait some for it to happen.
190       // OPENING below
191       while (true) {
192         int vid = ZKAssign.getVersion(this.watcher, REGIONINFO);
193         if (vid != versionid) {
194           versionid = vid;
195           break;
196         }
197       }
198       assertNotSame(-1, versionid);
199       // This uglyness below is what the openregionhandler on RS side does.
200       versionid = ZKAssign.transitionNode(server.getZooKeeper(), REGIONINFO,
201         SERVERNAME_A, EventType.M_ZK_REGION_OFFLINE,
202         EventType.RS_ZK_REGION_OPENING, versionid);
203       assertNotSame(-1, versionid);
204       // Move znode from OPENING to OPENED as RS does on successful open.
205       versionid = ZKAssign.transitionNodeOpened(this.watcher, REGIONINFO,
206         SERVERNAME_B, versionid);
207       assertNotSame(-1, versionid);
208       am.gate.set(false);
209       // Block here until our znode is cleared or until this test times out.
210       ZKAssign.blockUntilNoRIT(watcher);
211     } finally {
212       am.getExecutorService().shutdown();
213       am.shutdown();
214     }
215   }
216 
217   @Test(timeout = 5000)
218   public void testBalanceOnMasterFailoverScenarioWithClosedNode()
219       throws IOException, KeeperException, InterruptedException {
220     AssignmentManagerWithExtrasForTesting am =
221       setUpMockedAssignmentManager(this.server, this.serverManager);
222     try {
223       createRegionPlanAndBalance(am, SERVERNAME_A, SERVERNAME_B, REGIONINFO);
224       startFakeFailedOverMasterAssignmentManager(am, this.watcher);
225       while (!am.processRITInvoked) Thread.sleep(1);
226       // Now fake the region closing successfully over on the regionserver; the
227       // regionserver will have set the region in CLOSED state. This will
228       // trigger callback into AM. The below zk close call is from the RS close
229       // region handler duplicated here because its down deep in a private
230       // method hard to expose.
231       int versionid =
232         ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
233       assertNotSame(versionid, -1);
234       am.gate.set(false);
235       Mocking.waitForRegionOfflineInRIT(am, REGIONINFO.getEncodedName());
236 
237       // Get current versionid else will fail on transition from OFFLINE to
238       // OPENING below
239       while (true) {
240         int vid = ZKAssign.getVersion(this.watcher, REGIONINFO);
241         if (vid != versionid) {
242           versionid = vid;
243           break;
244         }
245       }
246       assertNotSame(-1, versionid);
247       // This uglyness below is what the openregionhandler on RS side does.
248       versionid = ZKAssign.transitionNode(server.getZooKeeper(), REGIONINFO,
249           SERVERNAME_A, EventType.M_ZK_REGION_OFFLINE,
250           EventType.RS_ZK_REGION_OPENING, versionid);
251       assertNotSame(-1, versionid);
252       // Move znode from OPENING to OPENED as RS does on successful open.
253       versionid = ZKAssign.transitionNodeOpened(this.watcher, REGIONINFO,
254           SERVERNAME_B, versionid);
255       assertNotSame(-1, versionid);
256 
257       // Block here until our znode is cleared or until this test timesout.
258       ZKAssign.blockUntilNoRIT(watcher);
259     } finally {
260       am.getExecutorService().shutdown();
261       am.shutdown();
262     }
263   }
264 
265   @Test(timeout = 5000)
266   public void testBalanceOnMasterFailoverScenarioWithOfflineNode()
267       throws IOException, KeeperException, InterruptedException {
268     AssignmentManagerWithExtrasForTesting am =
269       setUpMockedAssignmentManager(this.server, this.serverManager);
270     try {
271       createRegionPlanAndBalance(am, SERVERNAME_A, SERVERNAME_B, REGIONINFO);
272       startFakeFailedOverMasterAssignmentManager(am, this.watcher);
273       while (!am.processRITInvoked) Thread.sleep(1);
274       // Now fake the region closing successfully over on the regionserver; the
275       // regionserver will have set the region in CLOSED state. This will
276       // trigger callback into AM. The below zk close call is from the RS close
277       // region handler duplicated here because its down deep in a private
278       // method hard to expose.
279       int versionid =
280         ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
281       assertNotSame(versionid, -1);
282       Mocking.waitForRegionOfflineInRIT(am, REGIONINFO.getEncodedName());
283 
284       am.gate.set(false);
285       // Get current versionid else will fail on transition from OFFLINE to
286       // OPENING below
287       while (true) {
288         int vid = ZKAssign.getVersion(this.watcher, REGIONINFO);
289         if (vid != versionid) {
290           versionid = vid;
291           break;
292         }
293       }
294       assertNotSame(-1, versionid);
295       // This uglyness below is what the openregionhandler on RS side does.
296       versionid = ZKAssign.transitionNode(server.getZooKeeper(), REGIONINFO,
297           SERVERNAME_A, EventType.M_ZK_REGION_OFFLINE,
298           EventType.RS_ZK_REGION_OPENING, versionid);
299       assertNotSame(-1, versionid);
300       // Move znode from OPENING to OPENED as RS does on successful open.
301       versionid = ZKAssign.transitionNodeOpened(this.watcher, REGIONINFO,
302           SERVERNAME_B, versionid);
303       assertNotSame(-1, versionid);
304       // Block here until our znode is cleared or until this test timesout.
305       ZKAssign.blockUntilNoRIT(watcher);
306     } finally {
307       am.getExecutorService().shutdown();
308       am.shutdown();
309     }
310   }
311 
312   private void createRegionPlanAndBalance(final AssignmentManager am,
313       final ServerName from, final ServerName to, final HRegionInfo hri) {
314     // Call the balance function but fake the region being online first at
315     // servername from.
316     am.regionOnline(hri, from);
317     // Balance region from 'from' to 'to'. It calls unassign setting CLOSING state
318     // up in zk.  Create a plan and balance
319     am.balance(new RegionPlan(hri, from, to));
320   }
321 
322 
323   /**
324    * Tests AssignmentManager balance function.  Runs a balance moving a region
325    * from one server to another mocking regionserver responding over zk.
326    * @throws IOException
327    * @throws KeeperException
328    * @throws InterruptedException
329    */
330   @Test(timeout = 10000)
331   public void testBalance()
332   throws IOException, KeeperException, InterruptedException {
333     // Create and startup an executor.  This is used by AssignmentManager
334     // handling zk callbacks.
335     ExecutorService executor = startupMasterExecutor("testBalanceExecutor");
336 
337     // We need a mocked catalog tracker.
338     CatalogTracker ct = Mockito.mock(CatalogTracker.class);
339     LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(server
340         .getConfiguration());
341     // Create an AM.
342     AssignmentManager am = new AssignmentManager(this.server,
343         this.serverManager, ct, balancer, executor);
344     try {
345       // Make sure our new AM gets callbacks; once registered, can't unregister.
346       // Thats ok because we make a new zk watcher for each test.
347       this.watcher.registerListenerFirst(am);
348       // Call the balance function but fake the region being online first at
349       // SERVERNAME_A.  Create a balance plan.
350       am.regionOnline(REGIONINFO, SERVERNAME_A);
351       // Balance region from A to B.
352       RegionPlan plan = new RegionPlan(REGIONINFO, SERVERNAME_A, SERVERNAME_B);
353       am.balance(plan);
354 
355       // Now fake the region closing successfully over on the regionserver; the
356       // regionserver will have set the region in CLOSED state.  This will
357       // trigger callback into AM. The below zk close call is from the RS close
358       // region handler duplicated here because its down deep in a private
359       // method hard to expose.
360       int versionid =
361         ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
362       assertNotSame(versionid, -1);
363       // AM is going to notice above CLOSED and queue up a new assign.  The
364       // assign will go to open the region in the new location set by the
365       // balancer.  The zk node will be OFFLINE waiting for regionserver to
366       // transition it through OPENING, OPENED.  Wait till we see the RIT
367       // before we proceed.
368       Mocking.waitForRegionOfflineInRIT(am, REGIONINFO.getEncodedName());
369       // Get current versionid else will fail on transition from OFFLINE to OPENING below
370       while (true) {
371         int vid = ZKAssign.getVersion(this.watcher, REGIONINFO);
372         if (vid != versionid) {
373           versionid = vid;
374           break;
375         }
376       }
377       assertNotSame(-1, versionid);
378       // This uglyness below is what the openregionhandler on RS side does.
379       versionid = ZKAssign.transitionNode(server.getZooKeeper(), REGIONINFO,
380         SERVERNAME_A, EventType.M_ZK_REGION_OFFLINE,
381         EventType.RS_ZK_REGION_OPENING, versionid);
382       assertNotSame(-1, versionid);
383       // Move znode from OPENING to OPENED as RS does on successful open.
384       versionid =
385         ZKAssign.transitionNodeOpened(this.watcher, REGIONINFO, SERVERNAME_B, versionid);
386       assertNotSame(-1, versionid);
387       // Wait on the handler removing the OPENED znode.
388       while(am.isRegionInTransition(REGIONINFO) != null) Threads.sleep(1);
389     } finally {
390       executor.shutdown();
391       am.shutdown();
392       // Clean up all znodes
393       ZKAssign.deleteAllNodes(this.watcher);
394     }
395   }
396 
397   /**
398    * Run a simple server shutdown handler.
399    * @throws KeeperException
400    * @throws IOException
401    */
402   @Test
403   public void testShutdownHandler() throws KeeperException, IOException {
404     // Create and startup an executor.  This is used by AssignmentManager
405     // handling zk callbacks.
406     ExecutorService executor = startupMasterExecutor("testShutdownHandler");
407 
408     // We need a mocked catalog tracker.
409     CatalogTracker ct = Mockito.mock(CatalogTracker.class);
410     LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(server
411         .getConfiguration());
412     // Create an AM.
413     AssignmentManager am =
414       new AssignmentManager(this.server, this.serverManager, ct, balancer, executor);
415     try {
416       processServerShutdownHandler(ct, am, false, null);
417     } finally {
418       executor.shutdown();
419       am.shutdown();
420       // Clean up all znodes
421       ZKAssign.deleteAllNodes(this.watcher);
422     }
423   }
424 
425   /**
426    * To test closed region handler to remove rit and delete corresponding znode if region in pending
427    * close or closing while processing shutdown of a region server.(HBASE-5927).
428    * @throws KeeperException
429    * @throws IOException
430    */
431   @Test
432   public void testSSHWhenDisableTableInProgress()
433       throws KeeperException, IOException {
434     testCaseWithPartiallyDisabledState(TableState.DISABLING);
435     testCaseWithPartiallyDisabledState(TableState.DISABLED);
436   }
437 
438   /**
439    * To test if the split region is removed from RIT if the region was in SPLITTING state
440    * but the RS has actually completed the splitting in META but went down. See HBASE-6070
441    * and also HBASE-5806
442    * @throws KeeperException
443    * @throws IOException
444    */
445   @Test
446   public void testSSHWhenSplitRegionInProgress()
447       throws KeeperException, IOException, Exception {
448     // true indicates the region is split but still in RIT
449     testCaseWithSplitRegionPartial(true);
450     // false indicate the region is not split
451     testCaseWithSplitRegionPartial(false);
452 
453   }
454 
455   private void testCaseWithSplitRegionPartial(boolean regionSplitDone) throws KeeperException, IOException,
456       NodeExistsException, InterruptedException {
457     // Create and startup an executor. This is used by AssignmentManager
458     // handling zk callbacks.
459     ExecutorService executor = startupMasterExecutor("testSSHWhenSplitRegionInProgress");
460 
461     // We need a mocked catalog tracker.
462     CatalogTracker ct = Mockito.mock(CatalogTracker.class);
463     // Create an AM.
464     AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(this.server, this.serverManager);
465     // adding region to regions and servers maps.
466     am.regionOnline(REGIONINFO, SERVERNAME_A);
467     // adding region in pending close.
468     am.regionsInTransition.put(REGIONINFO.getEncodedName(), new RegionState(REGIONINFO,
469         State.SPLITTING, System.currentTimeMillis(), SERVERNAME_A));
470     am.getZKTable().setEnabledTable(REGIONINFO.getTableNameAsString());
471 
472     RegionTransitionData data = new RegionTransitionData(EventType.RS_ZK_REGION_SPLITTING,
473         REGIONINFO.getRegionName(), SERVERNAME_A);
474     String node = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName());
475     // create znode in M_ZK_REGION_CLOSING state.
476     ZKUtil.createAndWatch(this.watcher, node, data.getBytes());
477 
478     try {
479       processServerShutdownHandler(ct, am, regionSplitDone, null);
480       // check znode deleted or not.
481       // In both cases the znode should be deleted.
482 
483       if(regionSplitDone){
484         assertTrue("Region state of region in SPLITTING should be removed from rit.",
485             am.regionsInTransition.isEmpty());
486       }
487       else{
488         while (!am.assignInvoked) {
489           Thread.sleep(1);
490         }
491         assertTrue("Assign should be invoked.", am.assignInvoked);
492       }
493     } finally {
494       REGIONINFO.setOffline(false);
495       REGIONINFO.setSplit(false);
496       executor.shutdown();
497       am.shutdown();
498       // Clean up all znodes
499       ZKAssign.deleteAllNodes(this.watcher);
500     }
501   }
502 
503   private void testCaseWithPartiallyDisabledState(TableState state) throws KeeperException, IOException, NodeExistsException {
504     // Create and startup an executor. This is used by AssignmentManager
505     // handling zk callbacks.
506     ExecutorService executor = startupMasterExecutor("testSSHWhenDisableTableInProgress");
507 
508     // We need a mocked catalog tracker.
509     CatalogTracker ct = Mockito.mock(CatalogTracker.class);
510     LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(server.getConfiguration());
511     // Create an AM.
512     AssignmentManager am = new AssignmentManager(this.server, this.serverManager, ct, balancer,
513         executor);
514     // adding region to regions and servers maps.
515     am.regionOnline(REGIONINFO, SERVERNAME_A);
516     // adding region in pending close.
517     am.regionsInTransition.put(REGIONINFO.getEncodedName(), new RegionState(REGIONINFO,
518         State.PENDING_CLOSE, System.currentTimeMillis(), SERVERNAME_A));
519 
520     if (state == TableState.DISABLING) {
521       am.getZKTable().setDisablingTable(REGIONINFO.getTableNameAsString());
522     } else {
523       am.getZKTable().setDisabledTable(REGIONINFO.getTableNameAsString());
524     }
525 
526     RegionTransitionData data = new RegionTransitionData(EventType.M_ZK_REGION_CLOSING,
527         REGIONINFO.getRegionName(), SERVERNAME_A);
528     String node = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName());
529     // create znode in M_ZK_REGION_CLOSING state.
530     ZKUtil.createAndWatch(this.watcher, node, data.getBytes());
531 
532     try {
533       processServerShutdownHandler(ct, am, false, null);
534       // check znode deleted or not.
535       // In both cases the znode should be deleted.
536       assertTrue("The znode should be deleted.",ZKUtil.checkExists(this.watcher, node) == -1);
537       // check whether in rit or not.  In the DISABLING case also the below assert will be true
538       // but the piece of code added for HBASE-5927 will not do that.
539       if (state == TableState.DISABLED) {
540         assertTrue("Region state of region in pending close should be removed from rit.",
541             am.regionsInTransition.isEmpty());
542       }
543     } finally {
544       executor.shutdown();
545       am.shutdown();
546       // Clean up all znodes
547       ZKAssign.deleteAllNodes(this.watcher);
548     }
549   }
550 
551   private void processServerShutdownHandler(CatalogTracker ct, AssignmentManager am,
552     boolean splitRegion, ServerName sn)
553       throws IOException {
554     // Make sure our new AM gets callbacks; once registered, can't unregister.
555     // Thats ok because we make a new zk watcher for each test.
556     this.watcher.registerListenerFirst(am);
557     // Need to set up a fake scan of meta for the servershutdown handler
558     // Make an RS Interface implementation.  Make it so a scanner can go against it.
559     HRegionInterface implementation = Mockito.mock(HRegionInterface.class);
560     // Get a meta row result that has region up on SERVERNAME_A
561 
562     Result r = null;
563     if (sn == null) {
564       if (splitRegion) {
565         r = getMetaTableRowResultAsSplitRegion(REGIONINFO, SERVERNAME_A);
566       } else {
567         r = getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
568       }
569     } else {
570       if (sn.equals(SERVERNAME_A)) {
571         r = getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
572       } else if (sn.equals(SERVERNAME_B)) {
573         r = new Result(new KeyValue[0]);
574       }
575     }
576 
577     Mockito.when(implementation.openScanner((byte [])Mockito.any(), (Scan)Mockito.any())).
578       thenReturn(System.currentTimeMillis());
579     // Return a good result first and then return null to indicate end of scan
580     Mockito.when(implementation.next(Mockito.anyLong(), Mockito.anyInt())).
581       thenReturn(new Result [] {r}, (Result [])null);
582 
583     // Get a connection w/ mocked up common methods.
584     HConnection connection =
585       HConnectionTestingUtility.getMockedConnectionAndDecorate(HTU.getConfiguration(),
586         implementation, SERVERNAME_B, REGIONINFO);
587 
588     // Make it so we can get a catalogtracker from servermanager.. .needed
589     // down in guts of server shutdown handler.
590     Mockito.when(ct.getConnection()).thenReturn(connection);
591     Mockito.when(this.server.getCatalogTracker()).thenReturn(ct);
592 
593     // Now make a server shutdown handler instance and invoke process.
594     // Have it that SERVERNAME_A died.
595     DeadServer deadServers = new DeadServer();
596     deadServers.add(SERVERNAME_A);
597     // I need a services instance that will return the AM
598     MasterServices services = Mockito.mock(MasterServices.class);
599     Mockito.when(services.getAssignmentManager()).thenReturn(am);
600     Mockito.when(services.getZooKeeper()).thenReturn(this.watcher);
601     ServerShutdownHandler handler = null;
602     if (sn != null) {
603       handler = new ServerShutdownHandler(this.server, services, deadServers, sn, false);
604     } else {
605       handler = new ServerShutdownHandler(this.server, services, deadServers, SERVERNAME_A, false);
606     }
607     handler.process();
608     // The region in r will have been assigned.  It'll be up in zk as unassigned.
609   }
610 
611   /**
612    * @param sn ServerName to use making startcode and server in meta
613    * @param hri Region to serialize into HRegionInfo
614    * @return A mocked up Result that fakes a Get on a row in the
615    * <code>.META.</code> table.
616    * @throws IOException
617    */
618   private Result getMetaTableRowResult(final HRegionInfo hri,
619       final ServerName sn)
620   throws IOException {
621     // TODO: Move to a utilities class.  More than one test case can make use
622     // of this facility.
623     List<KeyValue> kvs = new ArrayList<KeyValue>();
624     kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
625       HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
626       Writables.getBytes(hri)));
627     kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
628       HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
629       Bytes.toBytes(sn.getHostAndPort())));
630     kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
631       HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
632       Bytes.toBytes(sn.getStartcode())));
633     return new Result(kvs);
634   }
635 
636   /**
637    * @param sn ServerName to use making startcode and server in meta
638    * @param hri Region to serialize into HRegionInfo
639    * @return A mocked up Result that fakes a Get on a row in the
640    * <code>.META.</code> table.
641    * @throws IOException
642    */
643   private Result getMetaTableRowResultAsSplitRegion(final HRegionInfo hri, final ServerName sn)
644       throws IOException {
645     hri.setOffline(true);
646     hri.setSplit(true);
647     return getMetaTableRowResult(hri, sn);
648   }
649 
650   /**
651    * Create and startup executor pools. Start same set as master does (just
652    * run a few less).
653    * @param name Name to give our executor
654    * @return Created executor (be sure to call shutdown when done).
655    */
656   private ExecutorService startupMasterExecutor(final String name) {
657     // TODO: Move up into HBaseTestingUtility?  Generally useful.
658     ExecutorService executor = new ExecutorService(name);
659     executor.startExecutorService(ExecutorType.MASTER_OPEN_REGION, 3);
660     executor.startExecutorService(ExecutorType.MASTER_CLOSE_REGION, 3);
661     executor.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS, 3);
662     executor.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS, 3);
663     return executor;
664   }
665 
666   @Test
667   public void testUnassignWithSplitAtSameTime() throws KeeperException, IOException {
668     // Region to use in test.
669     final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
670     // First amend the servermanager mock so that when we do send close of the
671     // first meta region on SERVERNAME_A, it will return true rather than
672     // default null.
673     Mockito.when(this.serverManager.sendRegionClose(SERVERNAME_A, hri, -1)).thenReturn(true);
674     // Need a mocked catalog tracker.
675     CatalogTracker ct = Mockito.mock(CatalogTracker.class);
676     LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(server
677         .getConfiguration());
678     // Create an AM.
679     AssignmentManager am =
680       new AssignmentManager(this.server, this.serverManager, ct, balancer, null);
681     try {
682       // First make sure my mock up basically works.  Unassign a region.
683       unassign(am, SERVERNAME_A, hri);
684       // This delete will fail if the previous unassign did wrong thing.
685       ZKAssign.deleteClosingNode(this.watcher, hri);
686       // Now put a SPLITTING region in the way.  I don't have to assert it
687       // go put in place.  This method puts it in place then asserts it still
688       // owns it by moving state from SPLITTING to SPLITTING.
689       int version = createNodeSplitting(this.watcher, hri, SERVERNAME_A);
690       // Now, retry the unassign with the SPLTTING in place.  It should just
691       // complete without fail; a sort of 'silent' recognition that the
692       // region to unassign has been split and no longer exists: TOOD: what if
693       // the split fails and the parent region comes back to life?
694       unassign(am, SERVERNAME_A, hri);
695       // This transition should fail if the znode has been messed with.
696       ZKAssign.transitionNode(this.watcher, hri, SERVERNAME_A,
697         EventType.RS_ZK_REGION_SPLITTING, EventType.RS_ZK_REGION_SPLITTING, version);
698       assertTrue(am.isRegionInTransition(hri) == null);
699     } finally {
700       am.shutdown();
701     }
702   }
703 
704   /**
705    * Tests the processDeadServersAndRegionsInTransition should not fail with NPE
706    * when it failed to get the children. Let's abort the system in this
707    * situation
708    * @throws ServiceException
709    */
710   @Test(timeout = 5000)
711   public void testProcessDeadServersAndRegionsInTransitionShouldNotFailWithNPE()
712       throws IOException, KeeperException, InterruptedException, ServiceException {
713     final RecoverableZooKeeper recoverableZk = Mockito
714         .mock(RecoverableZooKeeper.class);
715     AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(
716         this.server, this.serverManager);
717     Watcher zkw = new ZooKeeperWatcher(HBaseConfiguration.create(), "unittest",
718         null) {
719       public RecoverableZooKeeper getRecoverableZooKeeper() {
720         return recoverableZk;
721       }
722     };
723     ((ZooKeeperWatcher) zkw).registerListener(am);
724     Mockito.doThrow(new InterruptedException()).when(recoverableZk)
725         .getChildren("/hbase/unassigned", zkw);
726     am.setWatcher((ZooKeeperWatcher) zkw);
727     try {
728       am.processDeadServersAndRegionsInTransition();
729       fail("Expected to abort");
730     } catch (NullPointerException e) {
731       fail("Should not throw NPE");
732     } catch (RuntimeException e) {
733       assertEquals("Aborted", e.getLocalizedMessage());
734     }
735   }
736 
737   /**
738    * Creates a new ephemeral node in the SPLITTING state for the specified region.
739    * Create it ephemeral in case regionserver dies mid-split.
740    *
741    * <p>Does not transition nodes from other states.  If a node already exists
742    * for this region, a {@link NodeExistsException} will be thrown.
743    *
744    * @param zkw zk reference
745    * @param region region to be created as offline
746    * @param serverName server event originates from
747    * @return Version of znode created.
748    * @throws KeeperException
749    * @throws IOException
750    */
751   // Copied from SplitTransaction rather than open the method over there in
752   // the regionserver package.
753   private static int createNodeSplitting(final ZooKeeperWatcher zkw,
754       final HRegionInfo region, final ServerName serverName)
755   throws KeeperException, IOException {
756     RegionTransitionData data =
757       new RegionTransitionData(EventType.RS_ZK_REGION_SPLITTING,
758         region.getRegionName(), serverName);
759 
760     String node = ZKAssign.getNodeName(zkw, region.getEncodedName());
761     if (!ZKUtil.createEphemeralNodeAndWatch(zkw, node, data.getBytes())) {
762       throw new IOException("Failed create of ephemeral " + node);
763     }
764     // Transition node from SPLITTING to SPLITTING and pick up version so we
765     // can be sure this znode is ours; version is needed deleting.
766     return transitionNodeSplitting(zkw, region, serverName, -1);
767   }
768 
769   // Copied from SplitTransaction rather than open the method over there in
770   // the regionserver package.
771   private static int transitionNodeSplitting(final ZooKeeperWatcher zkw,
772       final HRegionInfo parent,
773       final ServerName serverName, final int version)
774   throws KeeperException, IOException {
775     return ZKAssign.transitionNode(zkw, parent, serverName,
776       EventType.RS_ZK_REGION_SPLITTING, EventType.RS_ZK_REGION_SPLITTING, version);
777   }
778 
779   private void unassign(final AssignmentManager am, final ServerName sn,
780       final HRegionInfo hri) {
781     // Before I can unassign a region, I need to set it online.
782     am.regionOnline(hri, sn);
783     // Unassign region.
784     am.unassign(hri);
785   }
786 
787   /**
788    * Create an {@link AssignmentManagerWithExtrasForTesting} that has mocked
789    * {@link CatalogTracker} etc.
790    * @param server
791    * @param manager
792    * @return An AssignmentManagerWithExtras with mock connections, etc.
793    * @throws IOException
794    * @throws KeeperException
795    */
796   private AssignmentManagerWithExtrasForTesting setUpMockedAssignmentManager(final Server server,
797       final ServerManager manager)
798   throws IOException, KeeperException {
799     // We need a mocked catalog tracker. Its used by our AM instance.
800     CatalogTracker ct = Mockito.mock(CatalogTracker.class);
801     // Make an RS Interface implementation. Make it so a scanner can go against
802     // it and a get to return the single region, REGIONINFO, this test is
803     // messing with. Needed when "new master" joins cluster. AM will try and
804     // rebuild its list of user regions and it will also get the HRI that goes
805     // with an encoded name by doing a Get on .META.
806     HRegionInterface ri = Mockito.mock(HRegionInterface.class);
807     // Get a meta row result that has region up on SERVERNAME_A for REGIONINFO
808     Result[] result = null;
809     if (enabling) {
810       result = new Result[2];
811       result[0] = getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
812       result[1] = getMetaTableRowResult(REGIONINFO_2, SERVERNAME_A);
813     }
814     Result r = getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
815     Mockito.when(ri .openScanner((byte[]) Mockito.any(), (Scan) Mockito.any())).
816       thenReturn(System.currentTimeMillis());
817    if (enabling) {
818       Mockito.when(ri.next(Mockito.anyLong(), Mockito.anyInt())).thenReturn(result, result, result,
819           (Result[]) null);
820       // If a get, return the above result too for REGIONINFO_2
821       Mockito.when(ri.get((byte[]) Mockito.any(), (Get) Mockito.any())).thenReturn(
822           getMetaTableRowResult(REGIONINFO_2, SERVERNAME_A));
823     } else {
824       // Return good result 'r' first and then return null to indicate end of scan
825       Mockito.when(ri.next(Mockito.anyLong(), Mockito.anyInt())).thenReturn(new Result[] { r });
826       // If a get, return the above result too for REGIONINFO
827       Mockito.when(ri.get((byte[]) Mockito.any(), (Get) Mockito.any())).thenReturn(r);
828     }
829     // Get a connection w/ mocked up common methods.
830     HConnection connection = HConnectionTestingUtility.
831       getMockedConnectionAndDecorate(HTU.getConfiguration(), ri, SERVERNAME_B,
832         REGIONINFO);
833     // Make it so we can get the connection from our mocked catalogtracker
834     Mockito.when(ct.getConnection()).thenReturn(connection);
835     // Create and startup an executor. Used by AM handling zk callbacks.
836     ExecutorService executor = startupMasterExecutor("mockedAMExecutor");
837     this.balancer = LoadBalancerFactory.getLoadBalancer(server.getConfiguration());
838     AssignmentManagerWithExtrasForTesting am = new AssignmentManagerWithExtrasForTesting(
839         server, manager, ct, balancer, executor);
840     return am;
841   }
842 
843   /**
844    * TestCase verifies that the regionPlan is updated whenever a region fails to open
845    * and the master tries to process RS_ZK_FAILED_OPEN state.(HBASE-5546).
846    */
847   @Test
848   public void testRegionPlanIsUpdatedWhenRegionFailsToOpen() throws IOException, KeeperException,
849       ServiceException, InterruptedException {
850     this.server.getConfiguration().setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
851         MockedLoadBalancer.class, LoadBalancer.class);
852     AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(this.server,
853         this.serverManager);
854     try {
855       // Boolean variable used for waiting until randomAssignment is called and new
856       // plan is generated.
857       AtomicBoolean gate = new AtomicBoolean(false);
858       if (balancer instanceof MockedLoadBalancer) {
859         ((MockedLoadBalancer) balancer).setGateVariable(gate);
860       }
861       ZKAssign.createNodeOffline(this.watcher, REGIONINFO, SERVERNAME_A);
862       int v = ZKAssign.getVersion(this.watcher, REGIONINFO);
863       ZKAssign.transitionNode(this.watcher, REGIONINFO, SERVERNAME_A, EventType.M_ZK_REGION_OFFLINE,
864           EventType.RS_ZK_REGION_FAILED_OPEN, v);
865       String path = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName());
866       RegionState state = new RegionState(REGIONINFO, State.OPENING, System.currentTimeMillis(),
867           SERVERNAME_A);
868       am.regionsInTransition.put(REGIONINFO.getEncodedName(), state);
869       // a dummy plan inserted into the regionPlans. This plan is cleared and new one is formed
870       am.regionPlans.put(REGIONINFO.getEncodedName(), new RegionPlan(REGIONINFO, null, SERVERNAME_A));
871       RegionPlan regionPlan = am.regionPlans.get(REGIONINFO.getEncodedName());
872       List<ServerName> serverList = new ArrayList<ServerName>(2);
873       serverList.add(SERVERNAME_B);
874       Mockito.when(this.serverManager.getOnlineServersList()).thenReturn(serverList);
875       am.nodeDataChanged(path);
876       // here we are waiting until the random assignment in the load balancer is called.
877       while (!gate.get()) {
878         Thread.sleep(10);
879       }
880       // new region plan may take some time to get updated after random assignment is called and
881       // gate is set to true.
882       RegionPlan newRegionPlan = am.regionPlans.get(REGIONINFO.getEncodedName());
883       while (newRegionPlan == null) {
884         Thread.sleep(10);
885         newRegionPlan = am.regionPlans.get(REGIONINFO.getEncodedName());
886       }
887       // the new region plan created may contain the same RS as destination but it should
888       // be new plan.
889       assertNotSame("Same region plan should not come", regionPlan, newRegionPlan);
890       assertTrue("Destnation servers should be different.", !(regionPlan.getDestination().equals(
891         newRegionPlan.getDestination())));
892       Mocking.waitForRegionOfflineInRIT(am, REGIONINFO.getEncodedName());
893     } finally {
894       this.server.getConfiguration().setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
895         DefaultLoadBalancer.class, LoadBalancer.class);
896       am.shutdown();
897     }
898   }
899 
900   /**
901    * Test verifies whether assignment is skipped for regions of tables in DISABLING state during
902    * clean cluster startup. See HBASE-6281.
903    *
904    * @throws KeeperException
905    * @throws IOException
906    * @throws Exception
907    */
908   @Test
909   public void testDisablingTableRegionsAssignmentDuringCleanClusterStartup()
910       throws KeeperException, IOException, Exception {
911     this.server.getConfiguration().setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
912         MockedLoadBalancer.class, LoadBalancer.class);
913     Mockito.when(this.serverManager.getOnlineServers()).thenReturn(
914         new HashMap<ServerName, HServerLoad>(0));
915     List<ServerName> destServers = new ArrayList<ServerName>(1);
916     destServers.add(SERVERNAME_A);
917     Mockito.when(this.serverManager.getDrainingServersList()).thenReturn(destServers);
918     // To avoid cast exception in DisableTableHandler process.
919     //Server server = new HMaster(HTU.getConfiguration());
920     AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server,
921         this.serverManager);
922     AtomicBoolean gate = new AtomicBoolean(false);
923     if (balancer instanceof MockedLoadBalancer) {
924       ((MockedLoadBalancer) balancer).setGateVariable(gate);
925     }
926     try{
927       // set table in disabling state.
928       am.getZKTable().setDisablingTable(REGIONINFO.getTableNameAsString());
929       am.joinCluster();
930       // should not call retainAssignment if we get empty regions in assignAllUserRegions.
931       assertFalse(
932           "Assign should not be invoked for disabling table regions during clean cluster startup.",
933           gate.get());
934       // need to change table state from disabling to disabled.
935       assertTrue("Table should be disabled.",
936           am.getZKTable().isDisabledTable(REGIONINFO.getTableNameAsString()));
937     } finally {
938       this.server.getConfiguration().setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
939         DefaultLoadBalancer.class, LoadBalancer.class);
940       am.getZKTable().setEnabledTable(REGIONINFO.getTableNameAsString());
941       am.shutdown();
942     }
943   }
944 
945   /**
946    * Test verifies whether all the enabling table regions assigned only once during master startup.
947    * 
948    * @throws KeeperException
949    * @throws IOException
950    * @throws Exception
951    */
952   @Test
953   public void testMasterRestartWhenTableInEnabling() throws KeeperException, IOException, Exception {
954     enabling = true;
955     this.server.getConfiguration().setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
956         DefaultLoadBalancer.class, LoadBalancer.class);
957     Map<ServerName, HServerLoad> serverAndLoad = new HashMap<ServerName, HServerLoad>();
958     serverAndLoad.put(SERVERNAME_A, null);
959     Mockito.when(this.serverManager.getOnlineServers()).thenReturn(serverAndLoad);
960     Mockito.when(this.serverManager.isServerOnline(SERVERNAME_B)).thenReturn(false);
961     Mockito.when(this.serverManager.isServerOnline(SERVERNAME_A)).thenReturn(true);
962     HTU.getConfiguration().setInt(HConstants.MASTER_PORT, 0);
963     Server server = new HMaster(HTU.getConfiguration());
964     Whitebox.setInternalState(server, "serverManager", this.serverManager);
965     assignmentCount = 0;
966     AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server,
967         this.serverManager);
968     am.regionOnline(new HRegionInfo("t1".getBytes(), HConstants.EMPTY_START_ROW,
969         HConstants.EMPTY_END_ROW), SERVERNAME_A);
970     am.gate.set(false);
971     try {
972       // set table in enabling state.
973       am.getZKTable().setEnablingTable(REGIONINFO.getTableNameAsString());
974       ZKAssign.createNodeOffline(this.watcher, REGIONINFO_2, SERVERNAME_B);
975 
976       am.joinCluster();
977       while (!am.getZKTable().isEnabledTable(REGIONINFO.getTableNameAsString())) {
978         Thread.sleep(10);
979       }
980       assertEquals("Number of assignments should be equal.", 2, assignmentCount);
981       assertTrue("Table should be enabled.",
982           am.getZKTable().isEnabledTable(REGIONINFO.getTableNameAsString()));
983     } finally {
984       enabling = false;
985       am.getZKTable().setEnabledTable(REGIONINFO.getTableNameAsString());
986       am.shutdown();
987       ZKAssign.deleteAllNodes(this.watcher);
988       assignmentCount = 0;
989     }
990   }
991 
992 
993 
994   /**
995    * When region in transition if region server opening the region gone down then region assignment
996    * taking long time(Waiting for timeout monitor to trigger assign). HBASE-5396(HBASE-6060) fixes this
997    * scenario. This test case verifies whether SSH calling assign for the region in transition or not.
998    *
999    * @throws KeeperException
1000    * @throws IOException
1001    * @throws ServiceException
1002    */
1003   @Test
1004   public void testSSHWhenSourceRSandDestRSInRegionPlanGoneDown() throws KeeperException, IOException,
1005       ServiceException {
1006     testSSHWhenSourceRSandDestRSInRegionPlanGoneDown(true);
1007     testSSHWhenSourceRSandDestRSInRegionPlanGoneDown(false);
1008   }
1009 
1010   private void testSSHWhenSourceRSandDestRSInRegionPlanGoneDown(boolean regionInOffline)
1011       throws IOException, KeeperException, ServiceException {
1012     // We need a mocked catalog tracker.
1013     CatalogTracker ct = Mockito.mock(CatalogTracker.class);
1014     // Create an AM.
1015     AssignmentManagerWithExtrasForTesting am =
1016         setUpMockedAssignmentManager(this.server, this.serverManager);
1017     // adding region in pending open.
1018     if (regionInOffline) {
1019       ServerName MASTER_SERVERNAME = new ServerName("example.org", 1111, 1111);
1020       am.regionsInTransition.put(REGIONINFO.getEncodedName(), new RegionState(REGIONINFO,
1021           State.OFFLINE, System.currentTimeMillis(), MASTER_SERVERNAME));
1022     } else {
1023       am.regionsInTransition.put(REGIONINFO.getEncodedName(), new RegionState(REGIONINFO,
1024           State.OPENING, System.currentTimeMillis(), SERVERNAME_B));
1025     }
1026     // adding region plan
1027     am.regionPlans.put(REGIONINFO.getEncodedName(), new RegionPlan(REGIONINFO, SERVERNAME_A, SERVERNAME_B));
1028     am.getZKTable().setEnabledTable(REGIONINFO.getTableNameAsString());
1029 
1030     try {
1031       processServerShutdownHandler(ct, am, false, SERVERNAME_A);
1032       processServerShutdownHandler(ct, am, false, SERVERNAME_B);
1033       if(regionInOffline){
1034         assertFalse("Assign should not be invoked.", am.assignInvoked);
1035       } else {
1036         assertTrue("Assign should be invoked.", am.assignInvoked);
1037       }
1038     } finally {
1039       am.regionsInTransition.remove(REGIONINFO.getEncodedName());
1040       am.regionPlans.remove(REGIONINFO.getEncodedName());
1041     }
1042   }
1043 
1044   /**
1045    * Mocked load balancer class used in the testcase to make sure that the testcase waits until
1046    * random assignment is called and the gate variable is set to true.
1047    */
1048   public static class MockedLoadBalancer extends DefaultLoadBalancer {
1049     private AtomicBoolean gate;
1050 
1051     public void setGateVariable(AtomicBoolean gate) {
1052       this.gate = gate;
1053     }
1054 
1055     @Override
1056     public ServerName randomAssignment(List<ServerName> servers) {
1057       ServerName randomServerName = super.randomAssignment(servers);
1058       this.gate.set(true);
1059       return randomServerName;
1060     }
1061 
1062     @Override
1063     public Map<ServerName, List<HRegionInfo>> retainAssignment(
1064         Map<HRegionInfo, ServerName> regions, List<ServerName> servers) {
1065       this.gate.set(true);
1066       return super.retainAssignment(regions, servers);
1067     }
1068 
1069   }
1070 
1071   /**
1072    * An {@link AssignmentManager} with some extra facility used testing
1073    */
1074   class AssignmentManagerWithExtrasForTesting extends AssignmentManager {
1075     // Keep a reference so can give it out below in {@link #getExecutorService}
1076     private final ExecutorService es;
1077     // Ditto for ct
1078     private final CatalogTracker ct;
1079     boolean processRITInvoked = false;
1080     boolean assignInvoked = false;
1081     AtomicBoolean gate = new AtomicBoolean(true);
1082 
1083     public AssignmentManagerWithExtrasForTesting(final Server master,
1084         final ServerManager serverManager, final CatalogTracker catalogTracker,
1085         final LoadBalancer balancer, final ExecutorService service)
1086     throws KeeperException, IOException {
1087       super(master, serverManager, catalogTracker, balancer, service);
1088       this.es = service;
1089       this.ct = catalogTracker;
1090     }
1091 
1092     @Override
1093     boolean processRegionInTransition(String encodedRegionName,
1094         HRegionInfo regionInfo,
1095         Map<ServerName, List<Pair<HRegionInfo, Result>>> deadServers)
1096         throws KeeperException, IOException {
1097       this.processRITInvoked = true;
1098       return super.processRegionInTransition(encodedRegionName, regionInfo,
1099           deadServers);
1100     }
1101     @Override
1102     void processRegionsInTransition(final RegionTransitionData data,
1103         final HRegionInfo regionInfo,
1104         final Map<ServerName, List<Pair<HRegionInfo, Result>>> deadServers,
1105         final int expectedVersion) throws KeeperException {
1106       while (this.gate.get()) Threads.sleep(1);
1107       super.processRegionsInTransition(data, regionInfo, deadServers, expectedVersion);
1108     }
1109     
1110     @Override
1111     public void assign(HRegionInfo region, boolean setOfflineInZK, boolean forceNewPlan,
1112         boolean hijack) {
1113       if (enabling) {
1114         assignmentCount++;
1115         this.regionOnline(region, SERVERNAME_A);
1116       } else {
1117         assignInvoked = true;
1118         super.assign(region, setOfflineInZK, forceNewPlan, hijack);
1119       }
1120     }
1121     
1122     @Override
1123     public ServerName getRegionServerOfRegion(HRegionInfo hri) {
1124       return SERVERNAME_A;
1125     }
1126     
1127     /** reset the watcher */
1128     void setWatcher(ZooKeeperWatcher watcher) {
1129       this.watcher = watcher;
1130     }
1131 
1132     /**
1133      * @return ExecutorService used by this instance.
1134      */
1135     ExecutorService getExecutorService() {
1136       return this.es;
1137     }
1138 
1139     /**
1140      * @return CatalogTracker used by this AM (Its a mock).
1141      */
1142     CatalogTracker getCatalogTracker() {
1143       return this.ct;
1144     }
1145   }
1146 
1147   /**
1148    * Call joinCluster on the passed AssignmentManager.  Do it in a thread
1149    * so it runs independent of what all else is going on.  Try to simulate
1150    * an AM running insided a failed over master by clearing all in-memory
1151    * AM state first.
1152   */
1153   private void startFakeFailedOverMasterAssignmentManager(final AssignmentManager am,
1154       final ZooKeeperWatcher watcher) {
1155     // Make sure our new AM gets callbacks; once registered, we can't unregister.
1156     // Thats ok because we make a new zk watcher for each test.
1157     watcher.registerListenerFirst(am);
1158     Thread t = new Thread("RunAmJoinCluster") {
1159       public void run() {
1160         // Call the joinCluster function as though we were doing a master
1161         // failover at this point. It will stall just before we go to add
1162         // the RIT region to our RIT Map in AM at processRegionsInTransition.
1163         // First clear any inmemory state from AM so it acts like a new master
1164         // coming on line.
1165         am.regionsInTransition.clear();
1166         am.regionPlans.clear();
1167         try {
1168           am.joinCluster();
1169         } catch (IOException e) {
1170           throw new RuntimeException(e);
1171         } catch (KeeperException e) {
1172           throw new RuntimeException(e);
1173         } catch (InterruptedException e) {
1174           throw new RuntimeException(e);
1175         }
1176       };
1177     };
1178     t.start();
1179     while (!t.isAlive()) Threads.sleep(1);
1180   }
1181 
1182   @org.junit.Rule
1183   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
1184     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
1185 }