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.assertNotSame;
21
22 import org.apache.hadoop.hbase.exceptions.DeserializationException;
23 import org.apache.hadoop.hbase.HRegionInfo;
24 import org.apache.hadoop.hbase.RegionTransition;
25 import org.apache.hadoop.hbase.ServerName;
26 import org.apache.hadoop.hbase.executor.EventType;
27 import org.apache.hadoop.hbase.master.RegionState.State;
28 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
29 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
30 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
31 import org.apache.zookeeper.KeeperException;
32
33 /**
34 * Package scoped mocking utility.
35 */
36 public class Mocking {
37
38 static void waitForRegionFailedToCloseAndSetToPendingClose(
39 AssignmentManager am, HRegionInfo hri) throws InterruptedException {
40 // Since region server is fake, sendRegionClose will fail, and closing
41 // region will fail. For testing purpose, moving it back to pending close
42 boolean wait = true;
43 while (wait) {
44 RegionState state = am.getRegionStates().getRegionState(hri);
45 if (state != null && state.isFailedClose()){
46 am.getRegionStates().updateRegionState(hri, State.PENDING_CLOSE);
47 wait = false;
48 } else {
49 Thread.sleep(1);
50 }
51 }
52 }
53
54 static void waitForRegionPendingOpenInRIT(AssignmentManager am, String encodedName)
55 throws InterruptedException {
56 // We used to do a check like this:
57 //!Mocking.verifyRegionState(this.watcher, REGIONINFO, EventType.M_ZK_REGION_OFFLINE)) {
58 // There is a race condition with this: because we may do the transition to
59 // RS_ZK_REGION_OPENING before the RIT is internally updated. We need to wait for the
60 // RIT to be as we need it to be instead. This cannot happen in a real cluster as we
61 // update the RIT before sending the openRegion request.
62
63 boolean wait = true;
64 while (wait) {
65 RegionState state = am.getRegionStates()
66 .getRegionsInTransition().get(encodedName);
67 if (state != null && state.isPendingOpen()){
68 wait = false;
69 } else {
70 Thread.sleep(1);
71 }
72 }
73 }
74
75 /**
76 * Verifies that the specified region is in the specified state in ZooKeeper.
77 * <p>
78 * Returns true if region is in transition and in the specified state in
79 * ZooKeeper. Returns false if the region does not exist in ZK or is in
80 * a different state.
81 * <p>
82 * Method synchronizes() with ZK so will yield an up-to-date result but is
83 * a slow read.
84 * @param zkw
85 * @param region
86 * @param expectedState
87 * @return true if region exists and is in expected state
88 * @throws DeserializationException
89 */
90 static boolean verifyRegionState(ZooKeeperWatcher zkw, HRegionInfo region, EventType expectedState)
91 throws KeeperException, DeserializationException {
92 String encoded = region.getEncodedName();
93
94 String node = ZKAssign.getNodeName(zkw, encoded);
95 zkw.sync(node);
96
97 // Read existing data of the node
98 byte [] existingBytes = null;
99 try {
100 existingBytes = ZKUtil.getDataAndWatch(zkw, node);
101 } catch (KeeperException.NoNodeException nne) {
102 return false;
103 } catch (KeeperException e) {
104 throw e;
105 }
106 if (existingBytes == null) return false;
107 RegionTransition rt = RegionTransition.parseFrom(existingBytes);
108 return rt.getEventType().equals(expectedState);
109 }
110 }