1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.chaos.actions;
20
21 import java.io.IOException;
22 import java.util.ArrayList;
23 import java.util.Collection;
24 import java.util.HashSet;
25 import java.util.LinkedList;
26 import java.util.List;
27 import java.util.Set;
28
29 import org.apache.commons.lang.math.RandomUtils;
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.hbase.ClusterStatus;
32 import org.apache.hadoop.hbase.HBaseCluster;
33 import org.apache.hadoop.hbase.HRegionInfo;
34 import org.apache.hadoop.hbase.IntegrationTestingUtility;
35 import org.apache.hadoop.hbase.MiniHBaseCluster;
36 import org.apache.hadoop.hbase.ServerLoad;
37 import org.apache.hadoop.hbase.ServerName;
38 import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
39 import org.apache.hadoop.hbase.client.Admin;
40 import org.apache.hadoop.hbase.util.Bytes;
41 import org.slf4j.Logger;
42
43
44
45
46 public abstract class Action {
47
48 public static final String KILL_MASTER_TIMEOUT_KEY =
49 "hbase.chaosmonkey.action.killmastertimeout";
50 public static final String START_MASTER_TIMEOUT_KEY =
51 "hbase.chaosmonkey.action.startmastertimeout";
52 public static final String KILL_RS_TIMEOUT_KEY = "hbase.chaosmonkey.action.killrstimeout";
53 public static final String START_RS_TIMEOUT_KEY = "hbase.chaosmonkey.action.startrstimeout";
54 public static final String KILL_ZK_NODE_TIMEOUT_KEY =
55 "hbase.chaosmonkey.action.killzknodetimeout";
56 public static final String START_ZK_NODE_TIMEOUT_KEY =
57 "hbase.chaosmonkey.action.startzknodetimeout";
58 public static final String KILL_DATANODE_TIMEOUT_KEY =
59 "hbase.chaosmonkey.action.killdatanodetimeout";
60 public static final String START_DATANODE_TIMEOUT_KEY =
61 "hbase.chaosmonkey.action.startdatanodetimeout";
62 public static final String KILL_NAMENODE_TIMEOUT_KEY =
63 "hbase.chaosmonkey.action.killnamenodetimeout";
64 public static final String START_NAMENODE_TIMEOUT_KEY =
65 "hbase.chaosmonkey.action.startnamenodetimeout";
66
67 protected static final long KILL_MASTER_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
68 protected static final long START_MASTER_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
69 protected static final long KILL_RS_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
70 protected static final long START_RS_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
71 protected static final long KILL_ZK_NODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
72 protected static final long START_ZK_NODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
73 protected static final long KILL_DATANODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
74 protected static final long START_DATANODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
75 protected static final long KILL_NAMENODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
76 protected static final long START_NAMENODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
77
78 protected ActionContext context;
79 protected HBaseCluster cluster;
80 protected ClusterStatus initialStatus;
81 protected ServerName[] initialServers;
82
83 protected long killMasterTimeout;
84 protected long startMasterTimeout;
85 protected long killRsTimeout;
86 protected long startRsTimeout;
87 protected long killZkNodeTimeout;
88 protected long startZkNodeTimeout;
89 protected long killDataNodeTimeout;
90 protected long startDataNodeTimeout;
91 protected long killNameNodeTimeout;
92 protected long startNameNodeTimeout;
93
94 public void init(ActionContext context) throws IOException {
95 this.context = context;
96 cluster = context.getHBaseCluster();
97 initialStatus = cluster.getInitialClusterStatus();
98 Collection<ServerName> regionServers = initialStatus.getServers();
99 initialServers = regionServers.toArray(new ServerName[regionServers.size()]);
100
101 killMasterTimeout = cluster.getConf().getLong(KILL_MASTER_TIMEOUT_KEY,
102 KILL_MASTER_TIMEOUT_DEFAULT);
103 startMasterTimeout = cluster.getConf().getLong(START_MASTER_TIMEOUT_KEY,
104 START_MASTER_TIMEOUT_DEFAULT);
105 killRsTimeout = cluster.getConf().getLong(KILL_RS_TIMEOUT_KEY, KILL_RS_TIMEOUT_DEFAULT);
106 startRsTimeout = cluster.getConf().getLong(START_RS_TIMEOUT_KEY, START_RS_TIMEOUT_DEFAULT);
107 killZkNodeTimeout = cluster.getConf().getLong(KILL_ZK_NODE_TIMEOUT_KEY,
108 KILL_ZK_NODE_TIMEOUT_DEFAULT);
109 startZkNodeTimeout = cluster.getConf().getLong(START_ZK_NODE_TIMEOUT_KEY,
110 START_ZK_NODE_TIMEOUT_DEFAULT);
111 killDataNodeTimeout = cluster.getConf().getLong(KILL_DATANODE_TIMEOUT_KEY,
112 KILL_DATANODE_TIMEOUT_DEFAULT);
113 startDataNodeTimeout = cluster.getConf().getLong(START_DATANODE_TIMEOUT_KEY,
114 START_DATANODE_TIMEOUT_DEFAULT);
115 killNameNodeTimeout =
116 cluster.getConf().getLong(KILL_NAMENODE_TIMEOUT_KEY, KILL_NAMENODE_TIMEOUT_DEFAULT);
117 startNameNodeTimeout =
118 cluster.getConf().getLong(START_NAMENODE_TIMEOUT_KEY, START_NAMENODE_TIMEOUT_DEFAULT);
119 }
120
121
122
123
124 protected abstract Logger getLogger();
125
126 public void perform() throws Exception { }
127
128
129 protected ServerName[] getCurrentServers() throws IOException {
130 ClusterStatus clusterStatus = cluster.getClusterStatus();
131 Collection<ServerName> regionServers = clusterStatus.getServers();
132 int count = regionServers == null ? 0 : regionServers.size();
133 if (count <= 0) {
134 return new ServerName [] {};
135 }
136 ServerName master = clusterStatus.getMaster();
137 Set<ServerName> masters = new HashSet<ServerName>();
138 masters.add(master);
139 masters.addAll(clusterStatus.getBackupMasters());
140 ArrayList<ServerName> tmp = new ArrayList<>(count);
141 tmp.addAll(regionServers);
142 tmp.removeAll(masters);
143 return tmp.toArray(new ServerName[0]);
144 }
145
146 protected void killMaster(ServerName server) throws IOException {
147 getLogger().info("Killing master:" + server);
148 cluster.killMaster(server);
149 cluster.waitForMasterToStop(server, killMasterTimeout);
150 getLogger().info("Killed master server:" + server);
151 }
152
153 protected void startMaster(ServerName server) throws IOException {
154 getLogger().info("Starting master:" + server.getHostname());
155 cluster.startMaster(server.getHostname(), server.getPort());
156 cluster.waitForActiveAndReadyMaster(startMasterTimeout);
157 getLogger().info("Started master: " + server);
158 }
159
160 protected void stopRs(ServerName server) throws IOException {
161 getLogger().info("Stopping regionserver " + server);
162 cluster.stopRegionServer(server);
163 cluster.waitForRegionServerToStop(server, killRsTimeout);
164 getLogger().info(String.format("Stopping regionserver %s. Reported num of rs: %s", server,
165 cluster.getClusterStatus().getLiveServersLoad().size()));
166 }
167
168 protected void suspendRs(ServerName server) throws IOException {
169 getLogger().info("Suspending regionserver %s" + server);
170 cluster.suspendRegionServer(server);
171 if(!(cluster instanceof MiniHBaseCluster)){
172 cluster.waitForRegionServerToStop(server, killRsTimeout);
173 }
174 getLogger().info(String.format("Suspending regionserver %s. Reported num of rs: %s", server,
175 cluster.getClusterStatus().getLiveServersLoad().size()));
176 }
177
178 protected void resumeRs(ServerName server) throws IOException {
179 getLogger().info("Resuming regionserver " + server);
180 cluster.resumeRegionServer(server);
181 if(!(cluster instanceof MiniHBaseCluster)){
182 cluster.waitForRegionServerToStart(server.getHostname(), server.getPort(), startRsTimeout);
183 }
184 getLogger().info(String.format("Resuming regionserver %s. Reported num of rs: %s", server,
185 cluster.getClusterStatus().getLiveServersLoad().size()));
186 }
187
188 protected void killRs(ServerName server) throws IOException {
189 getLogger().info("Killing regionserver " + server);
190 cluster.killRegionServer(server);
191 cluster.waitForRegionServerToStop(server, killRsTimeout);
192 getLogger().info(String.format("Killed regionserver %s. Reported num of rs: %s", server,
193 cluster.getClusterStatus().getLiveServersLoad().size()));
194 }
195
196 protected void startRs(ServerName server) throws IOException {
197 getLogger().info("Starting regionserver " + server.getAddress());
198 cluster.startRegionServer(server.getHostname(), server.getPort());
199 cluster.waitForRegionServerToStart(server.getHostname(), server.getPort(), startRsTimeout);
200 getLogger().info(String.format("Started regionserver %s. Reported num of rs: %s",
201 server.getAddress(), cluster.getClusterStatus().getLiveServersLoad().size()));
202 }
203
204 protected void killZKNode(ServerName server) throws IOException {
205 getLogger().info("Killing zookeeper node " + server);
206 cluster.killZkNode(server);
207 cluster.waitForZkNodeToStop(server, killZkNodeTimeout);
208 getLogger().info(String.format("Killed zookeeper node %s. Reported num of rs: %s", server,
209 cluster.getClusterStatus().getLiveServersLoad().size()));
210 }
211
212 protected void startZKNode(ServerName server) throws IOException {
213 getLogger().info("Starting zookeeper node " + server.getHostname());
214 cluster.startZkNode(server.getHostname(), server.getPort());
215 cluster.waitForZkNodeToStart(server, startZkNodeTimeout);
216 getLogger().info("Started zookeeper node " + server);
217 }
218
219 protected void killDataNode(ServerName server) throws IOException {
220 getLogger().info("Killing datanode " + server);
221 cluster.killDataNode(server);
222 cluster.waitForDataNodeToStop(server, killDataNodeTimeout);
223 getLogger().info(String.format("Killed datanode %s. Reported num of rs: %s", server,
224 cluster.getClusterStatus().getLiveServersLoad().size()));
225 }
226
227 protected void startDataNode(ServerName server) throws IOException {
228 getLogger().info("Starting datanode " + server.getHostname());
229 cluster.startDataNode(server);
230 cluster.waitForDataNodeToStart(server, startDataNodeTimeout);
231 getLogger().info("Started datanode " + server);
232 }
233
234 protected void killNameNode(ServerName server) throws IOException {
235 getLogger().info("Killing namenode : " + server.getHostname());
236 cluster.killNameNode(server);
237 cluster.waitForNameNodeToStop(server, killNameNodeTimeout);
238 getLogger().info("Killed namenode: " + server + ". Reported num of rs:"
239 + cluster.getClusterStatus().getServersSize());
240 }
241
242 protected void startNameNode(ServerName server) throws IOException {
243 getLogger().info("Starting Namenode : " + server.getHostname());
244 cluster.startNameNode(server);
245 cluster.waitForNameNodeToStart(server, startNameNodeTimeout);
246 getLogger().info("Started namenode: " + server);
247 }
248
249 protected void unbalanceRegions(ClusterStatus clusterStatus,
250 List<ServerName> fromServers, List<ServerName> toServers,
251 double fractionOfRegions) throws Exception {
252 List<byte[]> victimRegions = new LinkedList<byte[]>();
253 for (ServerName server : fromServers) {
254 ServerLoad serverLoad = clusterStatus.getLoad(server);
255
256 List<byte[]> regions = new LinkedList<byte[]>(serverLoad.getRegionsLoad().keySet());
257 int victimRegionCount = (int)Math.ceil(fractionOfRegions * regions.size());
258 getLogger().debug("Removing " + victimRegionCount + " regions from "
259 + server.getServerName());
260 for (int i = 0; i < victimRegionCount; ++i) {
261 int victimIx = RandomUtils.nextInt(regions.size());
262 String regionId = HRegionInfo.encodeRegionName(regions.remove(victimIx));
263 victimRegions.add(Bytes.toBytes(regionId));
264 }
265 }
266
267 getLogger().info("Moving " + victimRegions.size() + " regions from " + fromServers.size()
268 + " servers to " + toServers.size() + " different servers");
269 Admin admin = this.context.getHBaseIntegrationTestingUtility().getHBaseAdmin();
270 for (byte[] victimRegion : victimRegions) {
271
272
273 if (context.isStopping()) {
274 break;
275 }
276 int targetIx = RandomUtils.nextInt(toServers.size());
277 admin.move(victimRegion, Bytes.toBytes(toServers.get(targetIx).getServerName()));
278 }
279 }
280
281 protected void forceBalancer() throws Exception {
282 Admin admin = this.context.getHBaseIntegrationTestingUtility().getHBaseAdmin();
283 boolean result = false;
284 try {
285 result = admin.balancer();
286 } catch (Exception e) {
287 getLogger().warn("Got exception while doing balance ", e);
288 }
289 if (!result) {
290 getLogger().error("Balancer didn't succeed");
291 }
292 }
293
294 protected void setBalancer(boolean onOrOff, boolean synchronous) throws Exception {
295 Admin admin = this.context.getHBaseIntegrationTestingUtility().getHBaseAdmin();
296 try {
297 admin.setBalancerRunning(onOrOff, synchronous);
298 } catch (Exception e) {
299 getLogger().warn("Got exception while switching balance ", e);
300 }
301 }
302
303 public Configuration getConf() {
304 return cluster.getConf();
305 }
306
307
308
309
310 public static class ActionContext {
311 private IntegrationTestingUtility util;
312
313 public ActionContext(IntegrationTestingUtility util) {
314 this.util = util;
315 }
316
317 public IntegrationTestingUtility getHBaseIntegrationTestingUtility() {
318 return util;
319 }
320
321 public HBaseCluster getHBaseCluster() {
322 return util.getHBaseClusterInterface();
323 }
324
325 public boolean isStopping() {
326 return false;
327 }
328 }
329 }