View Javadoc

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.conf.Configuration;
34  import org.apache.hadoop.hbase.CellScannable;
35  import org.apache.hadoop.hbase.CellUtil;
36  import org.apache.hadoop.hbase.CoordinatedStateException;
37  import org.apache.hadoop.hbase.CoordinatedStateManager;
38  import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
39  import org.apache.hadoop.hbase.DoNotRetryIOException;
40  import org.apache.hadoop.hbase.HBaseConfiguration;
41  import org.apache.hadoop.hbase.HBaseTestingUtility;
42  import org.apache.hadoop.hbase.HConstants;
43  import org.apache.hadoop.hbase.HRegionInfo;
44  import org.apache.hadoop.hbase.MetaMockingUtil;
45  import org.apache.hadoop.hbase.RegionException;
46  import org.apache.hadoop.hbase.RegionTransition;
47  import org.apache.hadoop.hbase.Server;
48  import org.apache.hadoop.hbase.ServerLoad;
49  import org.apache.hadoop.hbase.ServerName;
50  import org.apache.hadoop.hbase.TableName;
51  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
52  import org.apache.hadoop.hbase.client.ClusterConnection;
53  import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
54  import org.apache.hadoop.hbase.client.Result;
55  import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
56  import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
57  import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
58  import org.apache.hadoop.hbase.coordination.ZkOpenRegionCoordination;
59  import org.apache.hadoop.hbase.exceptions.DeserializationException;
60  import org.apache.hadoop.hbase.executor.EventType;
61  import org.apache.hadoop.hbase.executor.ExecutorService;
62  import org.apache.hadoop.hbase.executor.ExecutorType;
63  import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
64  import org.apache.hadoop.hbase.master.RegionState.State;
65  import org.apache.hadoop.hbase.master.TableLockManager.NullTableLockManager;
66  import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
67  import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer;
68  import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
69  import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
70  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
71  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
72  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
73  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
74  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
75  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
76  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
77  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table;
78  import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
79  import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
80  import org.apache.hadoop.hbase.testclassification.MediumTests;
81  import org.apache.hadoop.hbase.util.Bytes;
82  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
83  import org.apache.hadoop.hbase.util.Threads;
84  import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
85  import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
86  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
87  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
88  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
89  import org.apache.zookeeper.KeeperException;
90  import org.apache.zookeeper.KeeperException.NodeExistsException;
91  import org.apache.zookeeper.Watcher;
92  import org.junit.After;
93  import org.junit.AfterClass;
94  import org.junit.Before;
95  import org.junit.BeforeClass;
96  import org.junit.Test;
97  import org.junit.experimental.categories.Category;
98  import org.mockito.Mockito;
99  import org.mockito.internal.util.reflection.Whitebox;
100 import org.mockito.invocation.InvocationOnMock;
101 import org.mockito.stubbing.Answer;
102 
103 import com.google.protobuf.RpcController;
104 import com.google.protobuf.ServiceException;
105 
106 
107 /**
108  * Test {@link AssignmentManager}
109  */
110 @Category(MediumTests.class)
111 public class TestAssignmentManager {
112   private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
113   private static final ServerName SERVERNAME_A =
114       ServerName.valueOf("example.org", 1234, 5678);
115   private static final ServerName SERVERNAME_B =
116       ServerName.valueOf("example.org", 0, 5678);
117   private static final HRegionInfo REGIONINFO =
118     new HRegionInfo(TableName.valueOf("t"),
119       HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
120   private static int assignmentCount;
121   private static boolean enabling = false;
122 
123   // Mocked objects or; get redone for each test.
124   private Server server;
125   private ServerManager serverManager;
126   private ZooKeeperWatcher watcher;
127   private CoordinatedStateManager cp;
128   private MetaTableLocator mtl;
129   private LoadBalancer balancer;
130   private HMaster master;
131   private ClusterConnection connection;
132 
133   @BeforeClass
134   public static void beforeClass() throws Exception {
135     HTU.getConfiguration().setBoolean("hbase.assignment.usezk", true);
136     HTU.startMiniZKCluster();
137   }
138 
139   @AfterClass
140   public static void afterClass() throws IOException {
141     HTU.shutdownMiniZKCluster();
142   }
143 
144   @Before
145   public void before() throws ZooKeeperConnectionException, IOException {
146     // TODO: Make generic versions of what we do below and put up in a mocking
147     // utility class or move up into HBaseTestingUtility.
148 
149     // Mock a Server.  Have it return a legit Configuration and ZooKeeperWatcher.
150     // If abort is called, be sure to fail the test (don't just swallow it
151     // silently as is mockito default).
152     this.server = Mockito.mock(Server.class);
153     Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("master,1,1"));
154     Mockito.when(server.getConfiguration()).thenReturn(HTU.getConfiguration());
155     this.watcher =
156       new ZooKeeperWatcher(HTU.getConfiguration(), "mockedServer", this.server, true);
157     Mockito.when(server.getZooKeeper()).thenReturn(this.watcher);
158     Mockito.doThrow(new RuntimeException("Aborted")).
159       when(server).abort(Mockito.anyString(), (Throwable)Mockito.anyObject());
160 
161     cp = new ZkCoordinatedStateManager();
162     cp.initialize(this.server);
163     cp.start();
164 
165     mtl = Mockito.mock(MetaTableLocator.class);
166 
167     Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp);
168     Mockito.when(server.getMetaTableLocator()).thenReturn(mtl);
169 
170     // Get a connection w/ mocked up common methods.
171     this.connection =
172       (ClusterConnection)HConnectionTestingUtility.getMockedConnection(HTU.getConfiguration());
173 
174     // Make it so we can get a catalogtracker from servermanager.. .needed
175     // down in guts of server shutdown handler.
176     Mockito.when(server.getConnection()).thenReturn(connection);
177     Mockito.when(connection.isManaged()).thenReturn(true);
178 
179     // Mock a ServerManager.  Say server SERVERNAME_{A,B} are online.  Also
180     // make it so if close or open, we return 'success'.
181     this.serverManager = Mockito.mock(ServerManager.class);
182     Mockito.when(this.serverManager.isServerOnline(SERVERNAME_A)).thenReturn(true);
183     Mockito.when(this.serverManager.isServerOnline(SERVERNAME_B)).thenReturn(true);
184     Mockito.when(this.serverManager.getDeadServers()).thenReturn(new DeadServer());
185     final Map<ServerName, ServerLoad> onlineServers = new HashMap<ServerName, ServerLoad>();
186     onlineServers.put(SERVERNAME_B, ServerLoad.EMPTY_SERVERLOAD);
187     onlineServers.put(SERVERNAME_A, ServerLoad.EMPTY_SERVERLOAD);
188     Mockito.when(this.serverManager.getOnlineServersList()).thenReturn(
189         new ArrayList<ServerName>(onlineServers.keySet()));
190     Mockito.when(this.serverManager.getOnlineServers()).thenReturn(onlineServers);
191 
192     List<ServerName> avServers = new ArrayList<ServerName>();
193     avServers.addAll(onlineServers.keySet());
194     Mockito.when(this.serverManager.createDestinationServersList()).thenReturn(avServers);
195     Mockito.when(this.serverManager.createDestinationServersList(null)).thenReturn(avServers);
196 
197     Mockito.when(this.serverManager.sendRegionClose(SERVERNAME_A, REGIONINFO, -1)).
198       thenReturn(true);
199     Mockito.when(this.serverManager.sendRegionClose(SERVERNAME_B, REGIONINFO, -1)).
200       thenReturn(true);
201     // Ditto on open.
202     Mockito.when(this.serverManager.sendRegionOpen(SERVERNAME_A, REGIONINFO, -1, null)).
203       thenReturn(RegionOpeningState.OPENED);
204     Mockito.when(this.serverManager.sendRegionOpen(SERVERNAME_B, REGIONINFO, -1, null)).
205       thenReturn(RegionOpeningState.OPENED);
206     this.master = Mockito.mock(HMaster.class);
207 
208     Mockito.when(this.master.getServerManager()).thenReturn(serverManager);
209   }
210 
211   @After public void after() throws KeeperException, IOException {
212     if (this.watcher != null) {
213       // Clean up all znodes
214       ZKAssign.deleteAllNodes(this.watcher);
215       this.watcher.close();
216       this.cp.stop();
217     }
218     if (this.connection != null) this.connection.close();
219   }
220 
221   /**
222    * Test a balance going on at same time as a master failover
223    *
224    * @throws IOException
225    * @throws KeeperException
226    * @throws InterruptedException
227    * @throws DeserializationException
228    */
229   @Test(timeout = 60000)
230   public void testBalanceOnMasterFailoverScenarioWithOpenedNode()
231       throws IOException, KeeperException, InterruptedException, ServiceException,
232       DeserializationException, CoordinatedStateException {
233     AssignmentManagerWithExtrasForTesting am =
234       setUpMockedAssignmentManager(this.server, this.serverManager);
235     try {
236       createRegionPlanAndBalance(am, SERVERNAME_A, SERVERNAME_B, REGIONINFO);
237       startFakeFailedOverMasterAssignmentManager(am, this.watcher);
238       while (!am.processRITInvoked) Thread.sleep(1);
239       // As part of the failover cleanup, the balancing region plan is removed.
240       // So a random server will be used to open the region. For testing purpose,
241       // let's assume it is going to open on server b:
242       am.addPlan(REGIONINFO.getEncodedName(), new RegionPlan(REGIONINFO, null, SERVERNAME_B));
243 
244       Mocking.waitForRegionFailedToCloseAndSetToPendingClose(am, REGIONINFO);
245 
246       // Now fake the region closing successfully over on the regionserver; the
247       // regionserver will have set the region in CLOSED state. This will
248       // trigger callback into AM. The below zk close call is from the RS close
249       // region handler duplicated here because its down deep in a private
250       // method hard to expose.
251       int versionid =
252         ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
253       assertNotSame(versionid, -1);
254       Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
255 
256       // Get current versionid else will fail on transition from OFFLINE to
257       // OPENING below
258       versionid = ZKAssign.getVersion(this.watcher, REGIONINFO);
259       assertNotSame(-1, versionid);
260       // This uglyness below is what the openregionhandler on RS side does.
261       versionid = ZKAssign.transitionNode(server.getZooKeeper(), REGIONINFO,
262         SERVERNAME_B, EventType.M_ZK_REGION_OFFLINE,
263         EventType.RS_ZK_REGION_OPENING, versionid);
264       assertNotSame(-1, versionid);
265       // Move znode from OPENING to OPENED as RS does on successful open.
266       versionid = ZKAssign.transitionNodeOpened(this.watcher, REGIONINFO,
267         SERVERNAME_B, versionid);
268       assertNotSame(-1, versionid);
269       am.gate.set(false);
270       // Block here until our znode is cleared or until this test times out.
271       ZKAssign.blockUntilNoRIT(watcher);
272     } finally {
273       am.getExecutorService().shutdown();
274       am.shutdown();
275     }
276   }
277 
278   @Test(timeout = 60000)
279   public void testBalanceOnMasterFailoverScenarioWithClosedNode()
280       throws IOException, KeeperException, InterruptedException, ServiceException,
281         DeserializationException, CoordinatedStateException {
282     AssignmentManagerWithExtrasForTesting am =
283       setUpMockedAssignmentManager(this.server, this.serverManager);
284     try {
285       createRegionPlanAndBalance(am, SERVERNAME_A, SERVERNAME_B, REGIONINFO);
286       startFakeFailedOverMasterAssignmentManager(am, this.watcher);
287       while (!am.processRITInvoked) Thread.sleep(1);
288       // As part of the failover cleanup, the balancing region plan is removed.
289       // So a random server will be used to open the region. For testing purpose,
290       // let's assume it is going to open on server b:
291       am.addPlan(REGIONINFO.getEncodedName(), new RegionPlan(REGIONINFO, null, SERVERNAME_B));
292 
293       Mocking.waitForRegionFailedToCloseAndSetToPendingClose(am, REGIONINFO);
294 
295       // Now fake the region closing successfully over on the regionserver; the
296       // regionserver will have set the region in CLOSED state. This will
297       // trigger callback into AM. The below zk close call is from the RS close
298       // region handler duplicated here because its down deep in a private
299       // method hard to expose.
300       int versionid =
301         ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
302       assertNotSame(versionid, -1);
303       am.gate.set(false);
304       Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
305 
306       // Get current versionid else will fail on transition from OFFLINE to
307       // OPENING below
308       versionid = ZKAssign.getVersion(this.watcher, REGIONINFO);
309       assertNotSame(-1, versionid);
310       // This uglyness below is what the openregionhandler on RS side does.
311       versionid = ZKAssign.transitionNode(server.getZooKeeper(), REGIONINFO,
312           SERVERNAME_B, EventType.M_ZK_REGION_OFFLINE,
313           EventType.RS_ZK_REGION_OPENING, versionid);
314       assertNotSame(-1, versionid);
315       // Move znode from OPENING to OPENED as RS does on successful open.
316       versionid = ZKAssign.transitionNodeOpened(this.watcher, REGIONINFO,
317           SERVERNAME_B, versionid);
318       assertNotSame(-1, versionid);
319 
320       // Block here until our znode is cleared or until this test timesout.
321       ZKAssign.blockUntilNoRIT(watcher);
322     } finally {
323       am.getExecutorService().shutdown();
324       am.shutdown();
325     }
326   }
327 
328   @Test(timeout = 60000)
329   public void testBalanceOnMasterFailoverScenarioWithOfflineNode()
330       throws IOException, KeeperException, InterruptedException, ServiceException,
331       DeserializationException, CoordinatedStateException {
332     AssignmentManagerWithExtrasForTesting am =
333       setUpMockedAssignmentManager(this.server, this.serverManager);
334     try {
335       createRegionPlanAndBalance(am, SERVERNAME_A, SERVERNAME_B, REGIONINFO);
336       startFakeFailedOverMasterAssignmentManager(am, this.watcher);
337       while (!am.processRITInvoked) Thread.sleep(1);
338       // As part of the failover cleanup, the balancing region plan is removed.
339       // So a random server will be used to open the region. For testing purpose,
340       // let's assume it is going to open on server b:
341       am.addPlan(REGIONINFO.getEncodedName(), new RegionPlan(REGIONINFO, null, SERVERNAME_B));
342 
343       Mocking.waitForRegionFailedToCloseAndSetToPendingClose(am, REGIONINFO);
344 
345       // Now fake the region closing successfully over on the regionserver; the
346       // regionserver will have set the region in CLOSED state. This will
347       // trigger callback into AM. The below zk close call is from the RS close
348       // region handler duplicated here because its down deep in a private
349       // method hard to expose.
350       int versionid =
351         ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
352       assertNotSame(versionid, -1);
353       Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
354 
355       am.gate.set(false);
356       // Get current versionid else will fail on transition from OFFLINE to
357       // OPENING below
358       versionid = ZKAssign.getVersion(this.watcher, REGIONINFO);
359       assertNotSame(-1, versionid);
360       // This uglyness below is what the openregionhandler on RS side does.
361       versionid = ZKAssign.transitionNode(server.getZooKeeper(), REGIONINFO,
362           SERVERNAME_B, EventType.M_ZK_REGION_OFFLINE,
363           EventType.RS_ZK_REGION_OPENING, versionid);
364       assertNotSame(-1, versionid);
365       // Move znode from OPENING to OPENED as RS does on successful open.
366       versionid = ZKAssign.transitionNodeOpened(this.watcher, REGIONINFO,
367           SERVERNAME_B, versionid);
368       assertNotSame(-1, versionid);
369       // Block here until our znode is cleared or until this test timesout.
370       ZKAssign.blockUntilNoRIT(watcher);
371     } finally {
372       am.getExecutorService().shutdown();
373       am.shutdown();
374     }
375   }
376 
377   private void createRegionPlanAndBalance(
378       final AssignmentManager am, final ServerName from,
379       final ServerName to, final HRegionInfo hri) throws RegionException {
380     // Call the balance function but fake the region being online first at
381     // servername from.
382     am.regionOnline(hri, from);
383     // Balance region from 'from' to 'to'. It calls unassign setting CLOSING state
384     // up in zk.  Create a plan and balance
385     am.balance(new RegionPlan(hri, from, to));
386   }
387 
388   /**
389    * Tests AssignmentManager balance function.  Runs a balance moving a region
390    * from one server to another mocking regionserver responding over zk.
391    * @throws IOException
392    * @throws KeeperException
393    * @throws DeserializationException
394    */
395   @Test (timeout=180000)
396   public void testBalance() throws IOException, KeeperException, DeserializationException,
397       InterruptedException, CoordinatedStateException {
398     // Create and startup an executor.  This is used by AssignmentManager
399     // handling zk callbacks.
400     ExecutorService executor = startupMasterExecutor("testBalanceExecutor");
401 
402     // We need a mocked catalog tracker.
403     LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(server
404         .getConfiguration());
405     // Create an AM.
406     AssignmentManager am = new AssignmentManager(this.server,
407       this.serverManager, balancer, executor, null, master.getTableLockManager());
408     am.failoverCleanupDone.set(true);
409     try {
410       // Make sure our new AM gets callbacks; once registered, can't unregister.
411       // Thats ok because we make a new zk watcher for each test.
412       this.watcher.registerListenerFirst(am);
413       // Call the balance function but fake the region being online first at
414       // SERVERNAME_A.  Create a balance plan.
415       am.regionOnline(REGIONINFO, SERVERNAME_A);
416       // Balance region from A to B.
417       RegionPlan plan = new RegionPlan(REGIONINFO, SERVERNAME_A, SERVERNAME_B);
418       am.balance(plan);
419 
420       RegionStates regionStates = am.getRegionStates();
421       // Must be failed to close since the server is fake
422       assertTrue(regionStates.isRegionInTransition(REGIONINFO)
423         && regionStates.isRegionInState(REGIONINFO, State.FAILED_CLOSE));
424       // Move it back to pending_close
425       regionStates.updateRegionState(REGIONINFO, State.PENDING_CLOSE);
426 
427       // Now fake the region closing successfully over on the regionserver; the
428       // regionserver will have set the region in CLOSED state.  This will
429       // trigger callback into AM. The below zk close call is from the RS close
430       // region handler duplicated here because its down deep in a private
431       // method hard to expose.
432       int versionid =
433         ZKAssign.transitionNodeClosed(this.watcher, REGIONINFO, SERVERNAME_A, -1);
434       assertNotSame(versionid, -1);
435       // AM is going to notice above CLOSED and queue up a new assign.  The
436       // assign will go to open the region in the new location set by the
437       // balancer.  The zk node will be OFFLINE waiting for regionserver to
438       // transition it through OPENING, OPENED.  Wait till we see the OFFLINE
439       // zk node before we proceed.
440       Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
441 
442       // Get current versionid else will fail on transition from OFFLINE to OPENING below
443       versionid = ZKAssign.getVersion(this.watcher, REGIONINFO);
444       assertNotSame(-1, versionid);
445       // This uglyness below is what the openregionhandler on RS side does.
446       versionid = ZKAssign.transitionNode(server.getZooKeeper(), REGIONINFO,
447         SERVERNAME_B, EventType.M_ZK_REGION_OFFLINE,
448         EventType.RS_ZK_REGION_OPENING, versionid);
449       assertNotSame(-1, versionid);
450       // Move znode from OPENING to OPENED as RS does on successful open.
451       versionid =
452         ZKAssign.transitionNodeOpened(this.watcher, REGIONINFO, SERVERNAME_B, versionid);
453       assertNotSame(-1, versionid);
454       // Wait on the handler removing the OPENED znode.
455       while(regionStates.isRegionInTransition(REGIONINFO)) Threads.sleep(1);
456     } finally {
457       executor.shutdown();
458       am.shutdown();
459       // Clean up all znodes
460       ZKAssign.deleteAllNodes(this.watcher);
461     }
462   }
463 
464   /**
465    * Run a simple server shutdown handler.
466    * @throws KeeperException
467    * @throws IOException
468    */
469   @Test (timeout=180000)
470   public void testShutdownHandler()
471       throws KeeperException, IOException, CoordinatedStateException, ServiceException {
472     // Create and startup an executor.  This is used by AssignmentManager
473     // handling zk callbacks.
474     ExecutorService executor = startupMasterExecutor("testShutdownHandler");
475 
476     // Create an AM.
477     AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(
478         this.server, this.serverManager);
479     try {
480       processServerShutdownHandler(am, false);
481     } finally {
482       executor.shutdown();
483       am.shutdown();
484       // Clean up all znodes
485       ZKAssign.deleteAllNodes(this.watcher);
486     }
487   }
488 
489   /**
490    * To test closed region handler to remove rit and delete corresponding znode
491    * if region in pending close or closing while processing shutdown of a region
492    * server.(HBASE-5927).
493    *
494    * @throws KeeperException
495    * @throws IOException
496    * @throws ServiceException
497    */
498   @Test (timeout=180000)
499   public void testSSHWhenDisableTableInProgress() throws KeeperException, IOException,
500     CoordinatedStateException, ServiceException {
501     testCaseWithPartiallyDisabledState(Table.State.DISABLING);
502     testCaseWithPartiallyDisabledState(Table.State.DISABLED);
503   }
504 
505 
506   /**
507    * To test if the split region is removed from RIT if the region was in SPLITTING state but the RS
508    * has actually completed the splitting in hbase:meta but went down. See HBASE-6070 and also HBASE-5806
509    *
510    * @throws KeeperException
511    * @throws IOException
512    */
513   @Test (timeout=180000)
514   public void testSSHWhenSplitRegionInProgress() throws KeeperException, IOException, Exception {
515     // true indicates the region is split but still in RIT
516     testCaseWithSplitRegionPartial(true);
517     // false indicate the region is not split
518     testCaseWithSplitRegionPartial(false);
519   }
520 
521   private void testCaseWithSplitRegionPartial(boolean regionSplitDone) throws KeeperException,
522       IOException, InterruptedException,
523     CoordinatedStateException, ServiceException {
524     // Create and startup an executor. This is used by AssignmentManager
525     // handling zk callbacks.
526     ExecutorService executor = startupMasterExecutor("testSSHWhenSplitRegionInProgress");
527     // We need a mocked catalog tracker.
528     ZKAssign.deleteAllNodes(this.watcher);
529 
530     // Create an AM.
531     AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(
532       this.server, this.serverManager);
533     // adding region to regions and servers maps.
534     am.regionOnline(REGIONINFO, SERVERNAME_A);
535     // adding region in pending close.
536     am.getRegionStates().updateRegionState(
537       REGIONINFO, State.SPLITTING, SERVERNAME_A);
538     am.getTableStateManager().setTableState(REGIONINFO.getTable(),
539       Table.State.ENABLED);
540     RegionTransition data = RegionTransition.createRegionTransition(EventType.RS_ZK_REGION_SPLITTING,
541         REGIONINFO.getRegionName(), SERVERNAME_A);
542     String node = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName());
543     // create znode in M_ZK_REGION_CLOSING state.
544     ZKUtil.createAndWatch(this.watcher, node, data.toByteArray());
545 
546     try {
547       processServerShutdownHandler(am, regionSplitDone);
548       // check znode deleted or not.
549       // In both cases the znode should be deleted.
550 
551       if (regionSplitDone) {
552         assertFalse("Region state of region in SPLITTING should be removed from rit.",
553             am.getRegionStates().isRegionsInTransition());
554       } else {
555         while (!am.assignInvoked) {
556           Thread.sleep(1);
557         }
558         assertTrue("Assign should be invoked.", am.assignInvoked);
559       }
560     } finally {
561       REGIONINFO.setOffline(false);
562       REGIONINFO.setSplit(false);
563       executor.shutdown();
564       am.shutdown();
565       // Clean up all znodes
566       ZKAssign.deleteAllNodes(this.watcher);
567     }
568   }
569 
570   private void testCaseWithPartiallyDisabledState(Table.State state) throws KeeperException,
571       IOException, CoordinatedStateException, ServiceException {
572     // Create and startup an executor. This is used by AssignmentManager
573     // handling zk callbacks.
574     ExecutorService executor = startupMasterExecutor("testSSHWhenDisableTableInProgress");
575     LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(server.getConfiguration());
576     ZKAssign.deleteAllNodes(this.watcher);
577 
578     // Create an AM.
579     AssignmentManager am = new AssignmentManager(this.server,
580       this.serverManager, balancer, executor, null, master.getTableLockManager());
581     // adding region to regions and servers maps.
582     am.regionOnline(REGIONINFO, SERVERNAME_A);
583     // adding region in pending close.
584     am.getRegionStates().updateRegionState(REGIONINFO, State.PENDING_CLOSE);
585     if (state == Table.State.DISABLING) {
586       am.getTableStateManager().setTableState(REGIONINFO.getTable(),
587         Table.State.DISABLING);
588     } else {
589       am.getTableStateManager().setTableState(REGIONINFO.getTable(),
590         Table.State.DISABLED);
591     }
592     RegionTransition data = RegionTransition.createRegionTransition(EventType.M_ZK_REGION_CLOSING,
593         REGIONINFO.getRegionName(), SERVERNAME_A);
594     // RegionTransitionData data = new
595     // RegionTransitionData(EventType.M_ZK_REGION_CLOSING,
596     // REGIONINFO.getRegionName(), SERVERNAME_A);
597     String node = ZKAssign.getNodeName(this.watcher, REGIONINFO.getEncodedName());
598     // create znode in M_ZK_REGION_CLOSING state.
599     ZKUtil.createAndWatch(this.watcher, node, data.toByteArray());
600 
601     try {
602       processServerShutdownHandler(am, false);
603       // check znode deleted or not.
604       // In both cases the znode should be deleted.
605       assertTrue("The znode should be deleted.", ZKUtil.checkExists(this.watcher, node) == -1);
606       // check whether in rit or not. In the DISABLING case also the below
607       // assert will be true but the piece of code added for HBASE-5927 will not
608       // do that.
609       if (state == Table.State.DISABLED) {
610         assertFalse("Region state of region in pending close should be removed from rit.",
611             am.getRegionStates().isRegionsInTransition());
612       }
613     } finally {
614       am.setEnabledTable(REGIONINFO.getTable());
615       executor.shutdown();
616       am.shutdown();
617       // Clean up all znodes
618       ZKAssign.deleteAllNodes(this.watcher);
619     }
620   }
621 
622   private void processServerShutdownHandler(AssignmentManager am, boolean splitRegion)
623       throws IOException, ServiceException {
624     // Make sure our new AM gets callbacks; once registered, can't unregister.
625     // Thats ok because we make a new zk watcher for each test.
626     this.watcher.registerListenerFirst(am);
627 
628     // Need to set up a fake scan of meta for the servershutdown handler
629     // Make an RS Interface implementation.  Make it so a scanner can go against it.
630     ClientProtos.ClientService.BlockingInterface implementation =
631       Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
632     // Get a meta row result that has region up on SERVERNAME_A
633 
634     Result r;
635     if (splitRegion) {
636       r = MetaMockingUtil.getMetaTableRowResultAsSplitRegion(REGIONINFO, SERVERNAME_A);
637     } else {
638       r = MetaMockingUtil.getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
639     }
640 
641     final ScanResponse.Builder builder = ScanResponse.newBuilder();
642     builder.setMoreResults(true);
643     builder.addCellsPerResult(r.size());
644     final List<CellScannable> cellScannables = new ArrayList<CellScannable>(1);
645     cellScannables.add(r);
646     Mockito.when(implementation.scan(
647       (RpcController)Mockito.any(), (ScanRequest)Mockito.any())).
648       thenAnswer(new Answer<ScanResponse>() {
649           @Override
650           public ScanResponse answer(InvocationOnMock invocation) throws Throwable {
651             PayloadCarryingRpcController controller = (PayloadCarryingRpcController) invocation
652                 .getArguments()[0];
653             if (controller != null) {
654               controller.setCellScanner(CellUtil.createCellScanner(cellScannables));
655             }
656             return builder.build();
657           }
658       });
659 
660     // Get a connection w/ mocked up common methods.
661     ClusterConnection connection =
662       HConnectionTestingUtility.getMockedConnectionAndDecorate(HTU.getConfiguration(),
663         null, implementation, SERVERNAME_B, REGIONINFO);
664     // These mocks were done up when all connections were managed.  World is different now we
665     // moved to unmanaged connections.  It messes up the intercepts done in these tests.
666     // Just mark connections as marked and then down in MetaTableAccessor, it will go the path
667     // that picks up the above mocked up 'implementation' so 'scans' of meta return the expected
668     // result.  Redo in new realm of unmanaged connections.
669     Mockito.when(connection.isManaged()).thenReturn(true);
670     try {
671       // Make it so we can get a catalogtracker from servermanager.. .needed
672       // down in guts of server shutdown handler.
673       Mockito.when(this.server.getConnection()).thenReturn(connection);
674 
675       // Now make a server shutdown handler instance and invoke process.
676       // Have it that SERVERNAME_A died.
677       DeadServer deadServers = new DeadServer();
678       deadServers.add(SERVERNAME_A);
679       // I need a services instance that will return the AM
680       MasterFileSystem fs = Mockito.mock(MasterFileSystem.class);
681       Mockito.doNothing().when(fs).setLogRecoveryMode();
682       Mockito.when(fs.getLogRecoveryMode()).thenReturn(RecoveryMode.LOG_REPLAY);
683       MasterServices services = Mockito.mock(MasterServices.class);
684       Mockito.when(services.getAssignmentManager()).thenReturn(am);
685       Mockito.when(services.getServerManager()).thenReturn(this.serverManager);
686       Mockito.when(services.getZooKeeper()).thenReturn(this.watcher);
687       Mockito.when(services.getMasterFileSystem()).thenReturn(fs);
688       Mockito.when(services.getConnection()).thenReturn(connection);
689       Configuration conf = server.getConfiguration();
690       Mockito.when(services.getConfiguration()).thenReturn(conf);
691       ServerShutdownHandler handler = new ServerShutdownHandler(this.server,
692           services, deadServers, SERVERNAME_A, false);
693       am.failoverCleanupDone.set(true);
694       handler.process();
695       // The region in r will have been assigned.  It'll be up in zk as unassigned.
696     } finally {
697       if (connection != null) connection.close();
698     }
699   }
700 
701   /**
702    * Create and startup executor pools. Start same set as master does (just
703    * run a few less).
704    * @param name Name to give our executor
705    * @return Created executor (be sure to call shutdown when done).
706    */
707   private ExecutorService startupMasterExecutor(final String name) {
708     // TODO: Move up into HBaseTestingUtility?  Generally useful.
709     ExecutorService executor = new ExecutorService(name);
710     executor.startExecutorService(ExecutorType.MASTER_OPEN_REGION, 3);
711     executor.startExecutorService(ExecutorType.MASTER_CLOSE_REGION, 3);
712     executor.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS, 3);
713     executor.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS, 3);
714     return executor;
715   }
716 
717   @Test (timeout=180000)
718   public void testUnassignWithSplitAtSameTime() throws KeeperException,
719       IOException, CoordinatedStateException {
720     // Region to use in test.
721     final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
722     // First amend the servermanager mock so that when we do send close of the
723     // first meta region on SERVERNAME_A, it will return true rather than
724     // default null.
725     Mockito.when(this.serverManager.sendRegionClose(SERVERNAME_A, hri, -1)).thenReturn(true);
726     // Need a mocked catalog tracker.
727     LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(server
728         .getConfiguration());
729     // Create an AM.
730     AssignmentManager am = new AssignmentManager(this.server,
731       this.serverManager, balancer, null, null, master.getTableLockManager());
732     try {
733       // First make sure my mock up basically works.  Unassign a region.
734       unassign(am, SERVERNAME_A, hri);
735       // This delete will fail if the previous unassign did wrong thing.
736       ZKAssign.deleteClosingNode(this.watcher, hri, SERVERNAME_A);
737       // Now put a SPLITTING region in the way.  I don't have to assert it
738       // go put in place.  This method puts it in place then asserts it still
739       // owns it by moving state from SPLITTING to SPLITTING.
740       int version = createNodeSplitting(this.watcher, hri, SERVERNAME_A);
741       // Now, retry the unassign with the SPLTTING in place.  It should just
742       // complete without fail; a sort of 'silent' recognition that the
743       // region to unassign has been split and no longer exists: TOOD: what if
744       // the split fails and the parent region comes back to life?
745       unassign(am, SERVERNAME_A, hri);
746       // This transition should fail if the znode has been messed with.
747       ZKAssign.transitionNode(this.watcher, hri, SERVERNAME_A,
748         EventType.RS_ZK_REGION_SPLITTING, EventType.RS_ZK_REGION_SPLITTING, version);
749       assertFalse(am.getRegionStates().isRegionInTransition(hri));
750     } finally {
751       am.shutdown();
752     }
753   }
754 
755   /**
756    * Tests the processDeadServersAndRegionsInTransition should not fail with NPE
757    * when it failed to get the children. Let's abort the system in this
758    * situation
759    * @throws ServiceException
760    */
761   @Test(timeout = 60000)
762   public void testProcessDeadServersAndRegionsInTransitionShouldNotFailWithNPE()
763       throws IOException, KeeperException, CoordinatedStateException,
764       InterruptedException, ServiceException {
765     final RecoverableZooKeeper recoverableZk = Mockito
766         .mock(RecoverableZooKeeper.class);
767     AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(
768       this.server, this.serverManager);
769     Watcher zkw = new ZooKeeperWatcher(HBaseConfiguration.create(), "unittest",
770         null) {
771       @Override
772       public RecoverableZooKeeper getRecoverableZooKeeper() {
773         return recoverableZk;
774       }
775     };
776     ((ZooKeeperWatcher) zkw).registerListener(am);
777     Mockito.doThrow(new InterruptedException()).when(recoverableZk)
778         .getChildren("/hbase/region-in-transition", null);
779     am.setWatcher((ZooKeeperWatcher) zkw);
780     try {
781       am.processDeadServersAndRegionsInTransition(null);
782       fail("Expected to abort");
783     } catch (NullPointerException e) {
784       fail("Should not throw NPE");
785     } catch (RuntimeException e) {
786       assertEquals("Aborted", e.getLocalizedMessage());
787     } finally {
788       am.shutdown();
789     }
790   }
791   /**
792    * TestCase verifies that the regionPlan is updated whenever a region fails to open
793    * and the master tries to process RS_ZK_FAILED_OPEN state.(HBASE-5546).
794    */
795   @Test(timeout = 60000)
796   public void testRegionPlanIsUpdatedWhenRegionFailsToOpen() throws IOException, KeeperException,
797       ServiceException, InterruptedException, CoordinatedStateException {
798     this.server.getConfiguration().setClass(
799       HConstants.HBASE_MASTER_LOADBALANCER_CLASS, MockedLoadBalancer.class,
800       LoadBalancer.class);
801     AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(
802       this.server, this.serverManager);
803     try {
804       // Boolean variable used for waiting until randomAssignment is called and
805       // new
806       // plan is generated.
807       AtomicBoolean gate = new AtomicBoolean(false);
808       if (balancer instanceof MockedLoadBalancer) {
809         ((MockedLoadBalancer) balancer).setGateVariable(gate);
810       }
811       ZKAssign.createNodeOffline(this.watcher, REGIONINFO, SERVERNAME_A);
812       int v = ZKAssign.getVersion(this.watcher, REGIONINFO);
813       ZKAssign.transitionNode(this.watcher, REGIONINFO, SERVERNAME_A,
814           EventType.M_ZK_REGION_OFFLINE, EventType.RS_ZK_REGION_FAILED_OPEN, v);
815       String path = ZKAssign.getNodeName(this.watcher, REGIONINFO
816           .getEncodedName());
817       am.getRegionStates().updateRegionState(
818         REGIONINFO, State.OPENING, SERVERNAME_A);
819       // a dummy plan inserted into the regionPlans. This plan is cleared and
820       // new one is formed
821       am.regionPlans.put(REGIONINFO.getEncodedName(), new RegionPlan(
822           REGIONINFO, null, SERVERNAME_A));
823       RegionPlan regionPlan = am.regionPlans.get(REGIONINFO.getEncodedName());
824       List<ServerName> serverList = new ArrayList<ServerName>(2);
825       serverList.add(SERVERNAME_B);
826       Mockito.when(
827           this.serverManager.createDestinationServersList(SERVERNAME_A))
828           .thenReturn(serverList);
829       am.nodeDataChanged(path);
830       // here we are waiting until the random assignment in the load balancer is
831       // called.
832       while (!gate.get()) {
833         Thread.sleep(10);
834       }
835       // new region plan may take some time to get updated after random
836       // assignment is called and
837       // gate is set to true.
838       RegionPlan newRegionPlan = am.regionPlans
839           .get(REGIONINFO.getEncodedName());
840       while (newRegionPlan == null) {
841         Thread.sleep(10);
842         newRegionPlan = am.regionPlans.get(REGIONINFO.getEncodedName());
843       }
844       // the new region plan created may contain the same RS as destination but
845       // it should
846       // be new plan.
847       assertNotSame("Same region plan should not come", regionPlan,
848           newRegionPlan);
849       assertTrue("Destination servers should be different.", !(regionPlan
850           .getDestination().equals(newRegionPlan.getDestination())));
851 
852       Mocking.waitForRegionPendingOpenInRIT(am, REGIONINFO.getEncodedName());
853     } finally {
854       this.server.getConfiguration().setClass(
855           HConstants.HBASE_MASTER_LOADBALANCER_CLASS, StochasticLoadBalancer.class,
856           LoadBalancer.class);
857       am.getExecutorService().shutdown();
858       am.shutdown();
859     }
860   }
861 
862   /**
863    * Mocked load balancer class used in the testcase to make sure that the testcase waits until
864    * random assignment is called and the gate variable is set to true.
865    */
866   public static class MockedLoadBalancer extends StochasticLoadBalancer {
867     private AtomicBoolean gate;
868 
869     public void setGateVariable(AtomicBoolean gate) {
870       this.gate = gate;
871     }
872 
873     @Override
874     public ServerName randomAssignment(HRegionInfo regionInfo, List<ServerName> servers) {
875       ServerName randomServerName = super.randomAssignment(regionInfo, servers);
876       this.gate.set(true);
877       return randomServerName;
878     }
879 
880     @Override
881     public Map<ServerName, List<HRegionInfo>> retainAssignment(
882         Map<HRegionInfo, ServerName> regions, List<ServerName> servers) {
883       this.gate.set(true);
884       return super.retainAssignment(regions, servers);
885     }
886   }
887 
888   /*
889    * Tests the scenario
890    * - a regionserver (SERVERNAME_A) owns a region (hence the meta would have
891    *   the SERVERNAME_A as the host for the region),
892    * - SERVERNAME_A goes down
893    * - one of the affected regions is assigned to a live regionserver (SERVERNAME_B) but that
894    *   assignment somehow fails. The region ends up in the FAILED_OPEN state on ZK
895    * - [Issue that the patch on HBASE-13330 fixes] when the master is restarted,
896    *   the SSH for SERVERNAME_A rightly thinks that the region is now on transition on
897    *   SERVERNAME_B. But the owner for the region is still SERVERNAME_A in the AM's states.
898    *   The AM thinks that the SSH for SERVERNAME_A will assign the region. The region remains
899    *   unassigned for ever.
900    */
901   @Test(timeout = 60000)
902   public void testAssignmentOfRegionInSSHAndInFailedOpenState() throws IOException,
903   KeeperException, ServiceException, CoordinatedStateException, InterruptedException {
904     AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(
905         this.server, this.serverManager);
906     ZKAssign.createNodeOffline(this.watcher, REGIONINFO, SERVERNAME_B);
907     int v = ZKAssign.getVersion(this.watcher, REGIONINFO);
908     ZKAssign.transitionNode(this.watcher, REGIONINFO, SERVERNAME_B,
909         EventType.M_ZK_REGION_OFFLINE, EventType.RS_ZK_REGION_FAILED_OPEN, v);
910     Mockito.when(this.serverManager.isServerOnline(SERVERNAME_B)).thenReturn(true);
911     Mockito.when(this.serverManager.isServerReachable(SERVERNAME_B)).thenReturn(true);
912     Mockito.when(this.serverManager.isServerOnline(SERVERNAME_A)).thenReturn(false);
913     DeadServer deadServers = new DeadServer();
914     deadServers.add(SERVERNAME_A);
915     Mockito.when(this.serverManager.getDeadServers()).thenReturn(deadServers);
916     final Map<ServerName, ServerLoad> onlineServers = new HashMap<ServerName, ServerLoad>();
917     onlineServers.put(SERVERNAME_B, ServerLoad.EMPTY_SERVERLOAD);
918     Mockito.when(this.serverManager.getOnlineServersList()).thenReturn(
919         new ArrayList<ServerName>(onlineServers.keySet()));
920     Mockito.when(this.serverManager.getOnlineServers()).thenReturn(onlineServers);
921     am.gate.set(false);
922     // join the cluster - that's when the AM is really kicking in after a restart
923     am.joinCluster();
924     while (!am.gate.get()) {
925       Thread.sleep(10);
926     }
927     assertTrue(am.getRegionStates().getRegionState(REGIONINFO).getState()
928         == RegionState.State.PENDING_OPEN);
929     am.shutdown();
930   }
931 
932   /**
933    * Test the scenario when the master is in failover and trying to process a
934    * region which is in Opening state on a dead RS. Master will force offline the
935    * region and put it in transition. AM relies on SSH to reassign it.
936    */
937   @Test(timeout = 60000)
938   public void testRegionInOpeningStateOnDeadRSWhileMasterFailover() throws IOException,
939       KeeperException, ServiceException, CoordinatedStateException, InterruptedException {
940     AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(
941       this.server, this.serverManager);
942     ZKAssign.createNodeOffline(this.watcher, REGIONINFO, SERVERNAME_A);
943     int version = ZKAssign.getVersion(this.watcher, REGIONINFO);
944     ZKAssign.transitionNode(this.watcher, REGIONINFO, SERVERNAME_A, EventType.M_ZK_REGION_OFFLINE,
945         EventType.RS_ZK_REGION_OPENING, version);
946     RegionTransition rt = RegionTransition.createRegionTransition(EventType.RS_ZK_REGION_OPENING,
947         REGIONINFO.getRegionName(), SERVERNAME_A, HConstants.EMPTY_BYTE_ARRAY);
948     version = ZKAssign.getVersion(this.watcher, REGIONINFO);
949     Mockito.when(this.serverManager.isServerOnline(SERVERNAME_A)).thenReturn(false);
950     am.getRegionStates().logSplit(SERVERNAME_A); // Assume log splitting is done
951     am.getRegionStates().createRegionState(REGIONINFO);
952     am.gate.set(false);
953 
954     BaseCoordinatedStateManager cp = new ZkCoordinatedStateManager();
955     cp.initialize(server);
956     cp.start();
957 
958     OpenRegionCoordination orc = cp.getOpenRegionCoordination();
959     ZkOpenRegionCoordination.ZkOpenRegionDetails zkOrd =
960       new ZkOpenRegionCoordination.ZkOpenRegionDetails();
961     zkOrd.setServerName(server.getServerName());
962     zkOrd.setVersion(version);
963 
964     assertFalse(am.processRegionsInTransition(rt, REGIONINFO, orc, zkOrd));
965     am.getTableStateManager().setTableState(REGIONINFO.getTable(), Table.State.ENABLED);
966     processServerShutdownHandler(am, false);
967     // Waiting for the assignment to get completed.
968     while (!am.gate.get()) {
969       Thread.sleep(10);
970     }
971     assertTrue("The region should be assigned immediately.", null != am.regionPlans.get(REGIONINFO
972         .getEncodedName()));
973     am.shutdown();
974   }
975 
976   /**
977    * Test verifies whether assignment is skipped for regions of tables in DISABLING state during
978    * clean cluster startup. See HBASE-6281.
979    *
980    * @throws KeeperException
981    * @throws IOException
982    * @throws Exception
983    */
984   @Test(timeout = 60000)
985   public void testDisablingTableRegionsAssignmentDuringCleanClusterStartup()
986       throws KeeperException, IOException, Exception {
987     this.server.getConfiguration().setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
988         MockedLoadBalancer.class, LoadBalancer.class);
989     Mockito.when(this.serverManager.getOnlineServers()).thenReturn(
990         new HashMap<ServerName, ServerLoad>(0));
991     List<ServerName> destServers = new ArrayList<ServerName>(1);
992     destServers.add(SERVERNAME_A);
993     Mockito.when(this.serverManager.createDestinationServersList()).thenReturn(destServers);
994     // To avoid cast exception in DisableTableHandler process.
995     HTU.getConfiguration().setInt(HConstants.MASTER_PORT, 0);
996 
997     CoordinatedStateManager csm = CoordinatedStateManagerFactory.getCoordinatedStateManager(
998       HTU.getConfiguration());
999     Server server = new HMaster(HTU.getConfiguration(), csm);
1000     AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server,
1001         this.serverManager);
1002 
1003     Whitebox.setInternalState(server, "metaTableLocator", Mockito.mock(MetaTableLocator.class));
1004 
1005     // Make it so we can get a catalogtracker from servermanager.. .needed
1006     // down in guts of server shutdown handler.
1007     Whitebox.setInternalState(server, "clusterConnection", am.getConnection());
1008 
1009     AtomicBoolean gate = new AtomicBoolean(false);
1010     if (balancer instanceof MockedLoadBalancer) {
1011       ((MockedLoadBalancer) balancer).setGateVariable(gate);
1012     }
1013     try{
1014       // set table in disabling state.
1015       am.getTableStateManager().setTableState(REGIONINFO.getTable(),
1016         Table.State.DISABLING);
1017       am.joinCluster();
1018       // should not call retainAssignment if we get empty regions in assignAllUserRegions.
1019       assertFalse(
1020           "Assign should not be invoked for disabling table regions during clean cluster startup.",
1021           gate.get());
1022       // need to change table state from disabling to disabled.
1023       assertTrue("Table should be disabled.",
1024           am.getTableStateManager().isTableState(REGIONINFO.getTable(),
1025             Table.State.DISABLED));
1026     } finally {
1027       this.server.getConfiguration().setClass(
1028         HConstants.HBASE_MASTER_LOADBALANCER_CLASS, StochasticLoadBalancer.class,
1029         LoadBalancer.class);
1030       am.getTableStateManager().setTableState(REGIONINFO.getTable(),
1031         Table.State.ENABLED);
1032       am.shutdown();
1033     }
1034   }
1035 
1036   /**
1037    * Test verifies whether all the enabling table regions assigned only once during master startup.
1038    *
1039    * @throws KeeperException
1040    * @throws IOException
1041    * @throws Exception
1042    */
1043   @Test (timeout=180000)
1044   public void testMasterRestartWhenTableInEnabling() throws KeeperException, IOException, Exception {
1045     enabling = true;
1046     List<ServerName> destServers = new ArrayList<ServerName>(1);
1047     destServers.add(SERVERNAME_A);
1048     Mockito.when(this.serverManager.createDestinationServersList()).thenReturn(destServers);
1049     Mockito.when(this.serverManager.isServerOnline(SERVERNAME_A)).thenReturn(true);
1050     HTU.getConfiguration().setInt(HConstants.MASTER_PORT, 0);
1051     CoordinatedStateManager csm = CoordinatedStateManagerFactory.getCoordinatedStateManager(
1052       HTU.getConfiguration());
1053     Server server = new HMaster(HTU.getConfiguration(), csm);
1054     Whitebox.setInternalState(server, "serverManager", this.serverManager);
1055     AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server,
1056         this.serverManager);
1057 
1058     Whitebox.setInternalState(server, "metaTableLocator", Mockito.mock(MetaTableLocator.class));
1059 
1060     // Make it so we can get a catalogtracker from servermanager.. .needed
1061     // down in guts of server shutdown handler.
1062     Whitebox.setInternalState(server, "clusterConnection", am.getConnection());
1063 
1064     try {
1065       // set table in enabling state.
1066       am.getTableStateManager().setTableState(REGIONINFO.getTable(),
1067         Table.State.ENABLING);
1068       new EnableTableHandler(server, REGIONINFO.getTable(),
1069           am, new NullTableLockManager(), true).prepare()
1070           .process();
1071       assertEquals("Number of assignments should be 1.", 1, assignmentCount);
1072       assertTrue("Table should be enabled.",
1073           am.getTableStateManager().isTableState(REGIONINFO.getTable(),
1074             Table.State.ENABLED));
1075     } finally {
1076       enabling = false;
1077       assignmentCount = 0;
1078       am.getTableStateManager().setTableState(REGIONINFO.getTable(),
1079         Table.State.ENABLED);
1080       am.shutdown();
1081       ZKAssign.deleteAllNodes(this.watcher);
1082     }
1083   }
1084 
1085   /**
1086    * Test verifies whether stale znodes of unknown tables as for the hbase:meta will be removed or
1087    * not.
1088    * @throws KeeperException
1089    * @throws IOException
1090    * @throws Exception
1091    */
1092   @Test (timeout=180000)
1093   public void testMasterRestartShouldRemoveStaleZnodesOfUnknownTableAsForMeta()
1094       throws Exception {
1095     List<ServerName> destServers = new ArrayList<ServerName>(1);
1096     destServers.add(SERVERNAME_A);
1097     Mockito.when(this.serverManager.createDestinationServersList()).thenReturn(destServers);
1098     Mockito.when(this.serverManager.isServerOnline(SERVERNAME_A)).thenReturn(true);
1099     HTU.getConfiguration().setInt(HConstants.MASTER_PORT, 0);
1100     CoordinatedStateManager csm = CoordinatedStateManagerFactory.getCoordinatedStateManager(
1101       HTU.getConfiguration());
1102     Server server = new HMaster(HTU.getConfiguration(), csm);
1103     Whitebox.setInternalState(server, "serverManager", this.serverManager);
1104     AssignmentManagerWithExtrasForTesting am = setUpMockedAssignmentManager(server,
1105         this.serverManager);
1106 
1107     Whitebox.setInternalState(server, "metaTableLocator", Mockito.mock(MetaTableLocator.class));
1108 
1109     // Make it so we can get a catalogtracker from servermanager.. .needed
1110     // down in guts of server shutdown handler.
1111     Whitebox.setInternalState(server, "clusterConnection", am.getConnection());
1112 
1113     try {
1114       TableName tableName = TableName.valueOf("dummyTable");
1115       // set table in enabling state.
1116       am.getTableStateManager().setTableState(tableName,
1117         Table.State.ENABLING);
1118       am.joinCluster();
1119       assertFalse("Table should not be present in zookeeper.",
1120         am.getTableStateManager().isTablePresent(tableName));
1121     } finally {
1122       am.shutdown();
1123     }
1124   }
1125   /**
1126    * When a region is in transition, if the region server opening the region goes down,
1127    * the region assignment takes a long time normally (waiting for timeout monitor to trigger assign).
1128    * This test is to make sure SSH reassigns it right away.
1129    */
1130   @Test (timeout=180000)
1131   public void testSSHTimesOutOpeningRegionTransition()
1132       throws KeeperException, IOException, CoordinatedStateException, ServiceException {
1133     // Create an AM.
1134     AssignmentManagerWithExtrasForTesting am =
1135       setUpMockedAssignmentManager(this.server, this.serverManager);
1136     // adding region in pending open.
1137     RegionState state = new RegionState(REGIONINFO,
1138       State.OPENING, System.currentTimeMillis(), SERVERNAME_A);
1139     am.getRegionStates().regionOnline(REGIONINFO, SERVERNAME_B);
1140     am.getRegionStates().regionsInTransition.put(REGIONINFO.getEncodedName(), state);
1141     // adding region plan
1142     am.regionPlans.put(REGIONINFO.getEncodedName(),
1143       new RegionPlan(REGIONINFO, SERVERNAME_B, SERVERNAME_A));
1144     am.getTableStateManager().setTableState(REGIONINFO.getTable(),
1145       Table.State.ENABLED);
1146 
1147     try {
1148       am.assignInvoked = false;
1149       processServerShutdownHandler(am, false);
1150       assertTrue(am.assignInvoked);
1151     } finally {
1152       am.getRegionStates().regionsInTransition.remove(REGIONINFO.getEncodedName());
1153       am.regionPlans.remove(REGIONINFO.getEncodedName());
1154       am.shutdown();
1155     }
1156   }
1157 
1158   /**
1159    * Scenario:<ul>
1160    *  <li> master starts a close, and creates a znode</li>
1161    *  <li> it fails just at this moment, before contacting the RS</li>
1162    *  <li> while the second master is coming up, the targeted RS dies. But it's before ZK timeout so
1163    *    we don't know, and we have an exception.</li>
1164    *  <li> the master must handle this nicely and reassign.
1165    *  </ul>
1166    */
1167   @Test (timeout=180000)
1168   public void testClosingFailureDuringRecovery() throws Exception {
1169 
1170     AssignmentManagerWithExtrasForTesting am =
1171         setUpMockedAssignmentManager(this.server, this.serverManager);
1172     ZKAssign.createNodeClosing(this.watcher, REGIONINFO, SERVERNAME_A);
1173     try {
1174       am.getRegionStates().createRegionState(REGIONINFO);
1175 
1176       assertFalse( am.getRegionStates().isRegionsInTransition() );
1177 
1178       am.processRegionInTransition(REGIONINFO.getEncodedName(), REGIONINFO);
1179 
1180       assertTrue( am.getRegionStates().isRegionsInTransition() );
1181     } finally {
1182       am.shutdown();
1183     }
1184   }
1185 
1186   /**
1187    * Creates a new ephemeral node in the SPLITTING state for the specified region.
1188    * Create it ephemeral in case regionserver dies mid-split.
1189    *
1190    * <p>Does not transition nodes from other states.  If a node already exists
1191    * for this region, a {@link NodeExistsException} will be thrown.
1192    *
1193    * @param zkw zk reference
1194    * @param region region to be created as offline
1195    * @param serverName server event originates from
1196    * @return Version of znode created.
1197    * @throws KeeperException
1198    * @throws IOException
1199    */
1200   // Copied from SplitTransaction rather than open the method over there in
1201   // the regionserver package.
1202   private static int createNodeSplitting(final ZooKeeperWatcher zkw,
1203       final HRegionInfo region, final ServerName serverName)
1204   throws KeeperException, IOException {
1205     RegionTransition rt =
1206       RegionTransition.createRegionTransition(EventType.RS_ZK_REGION_SPLITTING,
1207         region.getRegionName(), serverName);
1208 
1209     String node = ZKAssign.getNodeName(zkw, region.getEncodedName());
1210     if (!ZKUtil.createEphemeralNodeAndWatch(zkw, node, rt.toByteArray())) {
1211       throw new IOException("Failed create of ephemeral " + node);
1212     }
1213     // Transition node from SPLITTING to SPLITTING and pick up version so we
1214     // can be sure this znode is ours; version is needed deleting.
1215     return transitionNodeSplitting(zkw, region, serverName, -1);
1216   }
1217 
1218   // Copied from SplitTransaction rather than open the method over there in
1219   // the regionserver package.
1220   private static int transitionNodeSplitting(final ZooKeeperWatcher zkw,
1221       final HRegionInfo parent,
1222       final ServerName serverName, final int version)
1223   throws KeeperException, IOException {
1224     return ZKAssign.transitionNode(zkw, parent, serverName,
1225       EventType.RS_ZK_REGION_SPLITTING, EventType.RS_ZK_REGION_SPLITTING, version);
1226   }
1227 
1228   private void unassign(final AssignmentManager am, final ServerName sn,
1229       final HRegionInfo hri) throws RegionException {
1230     // Before I can unassign a region, I need to set it online.
1231     am.regionOnline(hri, sn);
1232     // Unassign region.
1233     am.unassign(hri);
1234   }
1235 
1236   /**
1237    * Create an {@link AssignmentManagerWithExtrasForTesting} that has mocked
1238    * {@link CatalogTracker} etc.
1239    * @param server
1240    * @param manager
1241    * @return An AssignmentManagerWithExtras with mock connections, etc.
1242    * @throws IOException
1243    * @throws KeeperException
1244    */
1245   private AssignmentManagerWithExtrasForTesting setUpMockedAssignmentManager(final Server server,
1246       final ServerManager manager) throws IOException, KeeperException,
1247         ServiceException, CoordinatedStateException {
1248     // Make an RS Interface implementation. Make it so a scanner can go against
1249     // it and a get to return the single region, REGIONINFO, this test is
1250     // messing with. Needed when "new master" joins cluster. AM will try and
1251     // rebuild its list of user regions and it will also get the HRI that goes
1252     // with an encoded name by doing a Get on hbase:meta
1253     ClientProtos.ClientService.BlockingInterface ri =
1254       Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
1255     // Get a meta row result that has region up on SERVERNAME_A for REGIONINFO
1256     Result r = MetaMockingUtil.getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
1257     final ScanResponse.Builder builder = ScanResponse.newBuilder();
1258     builder.setMoreResults(true);
1259     builder.addCellsPerResult(r.size());
1260     final List<CellScannable> rows = new ArrayList<CellScannable>(1);
1261     rows.add(r);
1262     Answer<ScanResponse> ans = new Answer<ClientProtos.ScanResponse>() {
1263       @Override
1264       public ScanResponse answer(InvocationOnMock invocation) throws Throwable {
1265         PayloadCarryingRpcController controller = (PayloadCarryingRpcController) invocation
1266             .getArguments()[0];
1267         if (controller != null) {
1268           controller.setCellScanner(CellUtil.createCellScanner(rows));
1269         }
1270         return builder.build();
1271       }
1272     };
1273     if (enabling) {
1274       Mockito.when(ri.scan((RpcController) Mockito.any(), (ScanRequest) Mockito.any()))
1275           .thenAnswer(ans).thenAnswer(ans).thenAnswer(ans).thenAnswer(ans).thenAnswer(ans)
1276           .thenReturn(ScanResponse.newBuilder().setMoreResults(false).build());
1277     } else {
1278       Mockito.when(ri.scan((RpcController) Mockito.any(), (ScanRequest) Mockito.any())).thenAnswer(
1279           ans);
1280     }
1281     // If a get, return the above result too for REGIONINFO
1282     GetResponse.Builder getBuilder = GetResponse.newBuilder();
1283     getBuilder.setResult(ProtobufUtil.toResult(r));
1284     Mockito.when(ri.get((RpcController)Mockito.any(), (GetRequest) Mockito.any())).
1285       thenReturn(getBuilder.build());
1286     // Get a connection w/ mocked up common methods.
1287     ClusterConnection connection = (ClusterConnection)HConnectionTestingUtility.
1288       getMockedConnectionAndDecorate(HTU.getConfiguration(), null,
1289         ri, SERVERNAME_B, REGIONINFO);
1290     Mockito.when(this.server.getConnection()).thenReturn(connection);
1291     // These mocks were done up when all connections were managed.  World is different now we
1292     // moved to unmanaged connections.  It messes up the intercepts done in these tests.
1293     // Just mark connections as marked and then down in MetaTableAccessor, it will go the path
1294     // that picks up the above mocked up 'implementation' so 'scans' of meta return the expected
1295     // result.  Redo in new realm of unmanaged connections.
1296     Mockito.when(connection.isManaged()).thenReturn(true);
1297     // Make it so we can get the connection from our mocked catalogtracker
1298     // Create and startup an executor. Used by AM handling zk callbacks.
1299     ExecutorService executor = startupMasterExecutor("mockedAMExecutor");
1300     this.balancer = LoadBalancerFactory.getLoadBalancer(server.getConfiguration());
1301     AssignmentManagerWithExtrasForTesting am = new AssignmentManagerWithExtrasForTesting(
1302       server, connection, manager, this.balancer, executor, new NullTableLockManager());
1303     return am;
1304   }
1305 
1306   /**
1307    * An {@link AssignmentManager} with some extra facility used testing
1308    */
1309   class AssignmentManagerWithExtrasForTesting extends AssignmentManager {
1310     // Keep a reference so can give it out below in {@link #getExecutorService}
1311     private final ExecutorService es;
1312     boolean processRITInvoked = false;
1313     boolean assignInvoked = false;
1314     AtomicBoolean gate = new AtomicBoolean(true);
1315     private ClusterConnection connection;
1316 
1317     public AssignmentManagerWithExtrasForTesting(
1318         final Server master, ClusterConnection connection, final ServerManager serverManager,
1319         final LoadBalancer balancer,
1320         final ExecutorService service, final TableLockManager tableLockManager)
1321             throws KeeperException, IOException, CoordinatedStateException {
1322       super(master, serverManager, balancer, service, null, tableLockManager);
1323       this.es = service;
1324       this.connection = connection;
1325     }
1326 
1327     @Override
1328     boolean processRegionInTransition(String encodedRegionName,
1329         HRegionInfo regionInfo) throws KeeperException, IOException {
1330       this.processRITInvoked = true;
1331       return super.processRegionInTransition(encodedRegionName, regionInfo);
1332     }
1333 
1334     @Override
1335     public void assign(HRegionInfo region, boolean setOfflineInZK, boolean forceNewPlan) {
1336       if (enabling) {
1337         assignmentCount++;
1338         this.regionOnline(region, SERVERNAME_A);
1339       } else {
1340         super.assign(region, setOfflineInZK, forceNewPlan);
1341         this.gate.set(true);
1342       }
1343     }
1344 
1345     @Override
1346     boolean assign(ServerName destination, List<HRegionInfo> regions)
1347         throws InterruptedException {
1348       if (enabling) {
1349         for (HRegionInfo region : regions) {
1350           assignmentCount++;
1351           this.regionOnline(region, SERVERNAME_A);
1352         }
1353         return true;
1354       }
1355       return super.assign(destination, regions);
1356     }
1357 
1358     @Override
1359     public void assign(List<HRegionInfo> regions)
1360         throws IOException, InterruptedException {
1361       assignInvoked = (regions != null && regions.size() > 0);
1362       super.assign(regions);
1363       this.gate.set(true);
1364     }
1365 
1366     /** reset the watcher */
1367     void setWatcher(ZooKeeperWatcher watcher) {
1368       this.watcher = watcher;
1369     }
1370 
1371     /**
1372      * @return ExecutorService used by this instance.
1373      */
1374     ExecutorService getExecutorService() {
1375       return this.es;
1376     }
1377 
1378     /*
1379      * Convenient method to retrieve mocked up connection
1380      */
1381     ClusterConnection getConnection() {
1382       return this.connection;
1383     }
1384 
1385     @Override
1386     public void shutdown() {
1387       super.shutdown();
1388       if (this.connection != null)
1389         try {
1390           this.connection.close();
1391         } catch (IOException e) {
1392           fail("Failed to close connection");
1393         }
1394     }
1395   }
1396 
1397   /**
1398    * Call joinCluster on the passed AssignmentManager.  Do it in a thread
1399    * so it runs independent of what all else is going on.  Try to simulate
1400    * an AM running insided a failed over master by clearing all in-memory
1401    * AM state first.
1402   */
1403   private void startFakeFailedOverMasterAssignmentManager(final AssignmentManager am,
1404       final ZooKeeperWatcher watcher) {
1405     // Make sure our new AM gets callbacks; once registered, we can't unregister.
1406     // Thats ok because we make a new zk watcher for each test.
1407     watcher.registerListenerFirst(am);
1408     Thread t = new Thread("RunAmJoinCluster") {
1409       @Override
1410       public void run() {
1411         // Call the joinCluster function as though we were doing a master
1412         // failover at this point. It will stall just before we go to add
1413         // the RIT region to our RIT Map in AM at processRegionsInTransition.
1414         // First clear any inmemory state from AM so it acts like a new master
1415         // coming on line.
1416         am.getRegionStates().regionsInTransition.clear();
1417         am.regionPlans.clear();
1418         try {
1419           am.joinCluster();
1420         } catch (IOException e) {
1421           throw new RuntimeException(e);
1422         } catch (KeeperException e) {
1423           throw new RuntimeException(e);
1424         } catch (InterruptedException e) {
1425           throw new RuntimeException(e);
1426         } catch (CoordinatedStateException e) {
1427           throw new RuntimeException(e);
1428         }
1429       }
1430     };
1431     t.start();
1432     while (!t.isAlive()) Threads.sleep(1);
1433   }
1434 
1435   @Test (timeout=180000)
1436   public void testForceAssignMergingRegion() throws Exception {
1437     // Region to use in test.
1438     final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
1439     // Need a mocked catalog tracker.
1440     LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(
1441       server.getConfiguration());
1442     // Create an AM.
1443     AssignmentManager am = new AssignmentManager(this.server,
1444       this.serverManager, balancer, null, null, master.getTableLockManager());
1445     RegionStates regionStates = am.getRegionStates();
1446     try {
1447       // First set the state of the region to merging
1448       regionStates.updateRegionState(hri, RegionState.State.MERGING);
1449       // Now, try to assign it with force new plan
1450       am.assign(hri, true, true);
1451       assertEquals("The region should be still in merging state",
1452         RegionState.State.MERGING, regionStates.getRegionState(hri).getState());
1453     } finally {
1454       am.shutdown();
1455     }
1456   }
1457 
1458   /**
1459    * Test assignment related ZK events are ignored by AM if the region is not known
1460    * by AM to be in transition. During normal operation, all assignments are started
1461    * by AM (not considering split/merge), if an event is received but the region
1462    * is not in transition, the event must be a very late one. So it can be ignored.
1463    * During master failover, since AM watches assignment znodes after failover cleanup
1464    * is completed, when an event comes in, AM should already have the region in transition
1465    * if ZK is used during the assignment action (only hbck doesn't use ZK for region
1466    * assignment). So during master failover, we can ignored such events too.
1467    */
1468   @Test (timeout=180000)
1469   public void testAssignmentEventIgnoredIfNotExpected() throws KeeperException, IOException,
1470       CoordinatedStateException {
1471     // Region to use in test.
1472     final HRegionInfo hri = HRegionInfo.FIRST_META_REGIONINFO;
1473     LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(
1474       server.getConfiguration());
1475     final AtomicBoolean zkEventProcessed = new AtomicBoolean(false);
1476     // Create an AM.
1477     AssignmentManager am = new AssignmentManager(this.server,
1478       this.serverManager, balancer, null, null, master.getTableLockManager()) {
1479 
1480       @Override
1481       void handleRegion(final RegionTransition rt, OpenRegionCoordination coordination,
1482                         OpenRegionCoordination.OpenRegionDetails ord) {
1483         super.handleRegion(rt, coordination, ord);
1484         if (rt != null && Bytes.equals(hri.getRegionName(),
1485           rt.getRegionName()) && rt.getEventType() == EventType.RS_ZK_REGION_OPENING) {
1486           zkEventProcessed.set(true);
1487         }
1488       }
1489     };
1490     try {
1491       // First make sure the region is not in transition
1492       am.getRegionStates().regionOffline(hri);
1493       zkEventProcessed.set(false); // Reset it before faking zk transition
1494       this.watcher.registerListenerFirst(am);
1495       assertFalse("The region should not be in transition",
1496         am.getRegionStates().isRegionInTransition(hri));
1497       ZKAssign.createNodeOffline(this.watcher, hri, SERVERNAME_A);
1498       // Trigger a transition event
1499       ZKAssign.transitionNodeOpening(this.watcher, hri, SERVERNAME_A);
1500       long startTime = EnvironmentEdgeManager.currentTime();
1501       while (!zkEventProcessed.get()) {
1502         assertTrue("Timed out in waiting for ZK event to be processed",
1503           EnvironmentEdgeManager.currentTime() - startTime < 30000);
1504         Threads.sleepWithoutInterrupt(100);
1505       }
1506       assertFalse(am.getRegionStates().isRegionInTransition(hri));
1507     } finally {
1508       am.shutdown();
1509     }
1510   }
1511 
1512   /**
1513    * If a table is deleted, we should not be able to balance it anymore.
1514    * Otherwise, the region will be brought back.
1515    * @throws Exception
1516    */
1517   @Test (timeout=180000)
1518   public void testBalanceRegionOfDeletedTable() throws Exception {
1519     AssignmentManager am = new AssignmentManager(this.server, this.serverManager,
1520       balancer, null, null, master.getTableLockManager());
1521     RegionStates regionStates = am.getRegionStates();
1522     HRegionInfo hri = REGIONINFO;
1523     regionStates.createRegionState(hri);
1524     assertFalse(regionStates.isRegionInTransition(hri));
1525     RegionPlan plan = new RegionPlan(hri, SERVERNAME_A, SERVERNAME_B);
1526     // Fake table is deleted
1527     regionStates.tableDeleted(hri.getTable());
1528     am.balance(plan);
1529     assertFalse("The region should not in transition",
1530       regionStates.isRegionInTransition(hri));
1531     am.shutdown();
1532   }
1533 
1534   /**
1535    * Tests an on-the-fly RPC that was scheduled for the earlier RS on the same port
1536    * for openRegion. AM should assign this somewhere else. (HBASE-9721)
1537    */
1538   @SuppressWarnings("unchecked")
1539   @Test (timeout=180000)
1540   public void testOpenCloseRegionRPCIntendedForPreviousServer() throws Exception {
1541     Mockito.when(this.serverManager.sendRegionOpen(Mockito.eq(SERVERNAME_B), Mockito.eq(REGIONINFO),
1542       Mockito.anyInt(), (List<ServerName>)Mockito.any()))
1543       .thenThrow(new DoNotRetryIOException());
1544     this.server.getConfiguration().setInt("hbase.assignment.maximum.attempts", 100);
1545 
1546     HRegionInfo hri = REGIONINFO;
1547     LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(
1548       server.getConfiguration());
1549     // Create an AM.
1550     AssignmentManager am = new AssignmentManager(this.server,
1551       this.serverManager, balancer, null, null, master.getTableLockManager());
1552     RegionStates regionStates = am.getRegionStates();
1553     try {
1554       am.regionPlans.put(REGIONINFO.getEncodedName(),
1555         new RegionPlan(REGIONINFO, null, SERVERNAME_B));
1556 
1557       // Should fail once, but succeed on the second attempt for the SERVERNAME_A
1558       am.assign(hri, true, false);
1559     } finally {
1560       assertEquals(SERVERNAME_A, regionStates.getRegionState(REGIONINFO).getServerName());
1561       am.shutdown();
1562     }
1563   }
1564 
1565   /**
1566    * Tests close region call on a region server that is aborting
1567    */
1568   @Test (timeout=180000)
1569   public void testCloseRegionOnAbortingRS() throws Exception {
1570     this.server.getConfiguration().setInt("hbase.assignment.maximum.attempts", 2);
1571 
1572     HRegionInfo hri = REGIONINFO;
1573     LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(
1574       server.getConfiguration());
1575     // Create an AM.
1576     AssignmentManager am = new AssignmentManager(this.server,
1577       this.serverManager, balancer, null, null, master.getTableLockManager());
1578     RegionStates regionStates = am.getRegionStates();
1579 
1580     regionStates.createRegionState(hri, State.OPEN, SERVERNAME_B, SERVERNAME_B);
1581 
1582     // mock aborting region server
1583     Mockito.when(this.serverManager.sendRegionClose(Mockito.eq(SERVERNAME_B), Mockito.eq(REGIONINFO),
1584       Mockito.anyInt(), (ServerName)Mockito.any(), Mockito.anyBoolean()))
1585       .thenThrow(new RegionServerAbortedException(""));
1586 
1587     // try to unassign the region
1588     am.unassign(hri);
1589 
1590     // assert that the we have FAILED_CLOSE for region state
1591     assertEquals(State.FAILED_CLOSE, regionStates.getRegionState(REGIONINFO).getState());
1592     assertEquals(SERVERNAME_B, regionStates.getRegionState(REGIONINFO).getServerName());
1593 
1594     am.shutdown();
1595   }
1596 
1597   /**
1598    * Tests close region call on a region server that is not in onlineServer list
1599    */
1600   @Test (timeout=180000)
1601   public void testCloseRegionOnServerNotOnline() throws Exception {
1602     this.server.getConfiguration().setInt("hbase.assignment.maximum.attempts", 2);
1603 
1604     HRegionInfo hri = REGIONINFO;
1605     LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(
1606       server.getConfiguration());
1607     // Create an AM.
1608     AssignmentManager am = new AssignmentManager(this.server,
1609       this.serverManager, balancer, null, null, master.getTableLockManager()) {
1610       @Override
1611       protected boolean wasRegionOnDeadServerByMeta(HRegionInfo region, ServerName sn) {
1612         return true;
1613       };
1614     };
1615     RegionStates regionStates = am.getRegionStates();
1616 
1617     regionStates.createRegionState(hri, State.OPEN, SERVERNAME_B, SERVERNAME_B);
1618 
1619     // mock that RS is expired, but not processed
1620     Mockito.when(this.serverManager.isServerOnline(SERVERNAME_B))
1621       .thenReturn(false);
1622 
1623     // try to unassign the region
1624     am.unassign(hri);
1625 
1626     // assert that the we have OFFLINE
1627     assertEquals(State.OFFLINE, regionStates.getRegionState(REGIONINFO).getState());
1628 
1629     // try to assign the region before SSH
1630     am.regionPlans.put(REGIONINFO.getEncodedName(),
1631       new RegionPlan(REGIONINFO, null, SERVERNAME_A));
1632     am.assign(hri, true, false);
1633 
1634     // assert that the we still have OFFLINE
1635     assertEquals(State.OFFLINE, regionStates.getRegionState(REGIONINFO).getState());
1636 
1637     am.shutdown();
1638   }
1639 }