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.LinkedList;
25 import java.util.List;
26
27 import org.apache.commons.lang.math.RandomUtils;
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
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.ServerLoad;
36 import org.apache.hadoop.hbase.ServerName;
37 import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
38 import org.apache.hadoop.hbase.client.Admin;
39 import org.apache.hadoop.hbase.util.Bytes;
40
41
42
43
44 public class Action {
45
46 public static final String KILL_MASTER_TIMEOUT_KEY =
47 "hbase.chaosmonkey.action.killmastertimeout";
48 public static final String START_MASTER_TIMEOUT_KEY =
49 "hbase.chaosmonkey.action.startmastertimeout";
50 public static final String KILL_RS_TIMEOUT_KEY = "hbase.chaosmonkey.action.killrstimeout";
51 public static final String START_RS_TIMEOUT_KEY = "hbase.chaosmonkey.action.startrstimeout";
52 public static final String KILL_ZK_NODE_TIMEOUT_KEY =
53 "hbase.chaosmonkey.action.killzknodetimeout";
54 public static final String START_ZK_NODE_TIMEOUT_KEY =
55 "hbase.chaosmonkey.action.startzknodetimeout";
56 public static final String KILL_DATANODE_TIMEOUT_KEY =
57 "hbase.chaosmonkey.action.killdatanodetimeout";
58 public static final String START_DATANODE_TIMEOUT_KEY =
59 "hbase.chaosmonkey.action.startdatanodetimeout";
60
61 protected static final Log LOG = LogFactory.getLog(Action.class);
62
63 protected static final long KILL_MASTER_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
64 protected static final long START_MASTER_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
65 protected static final long KILL_RS_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
66 protected static final long START_RS_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
67 protected static final long KILL_ZK_NODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
68 protected static final long START_ZK_NODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
69 protected static final long KILL_DATANODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
70 protected static final long START_DATANODE_TIMEOUT_DEFAULT = PolicyBasedChaosMonkey.TIMEOUT;
71
72 protected ActionContext context;
73 protected HBaseCluster cluster;
74 protected ClusterStatus initialStatus;
75 protected ServerName[] initialServers;
76
77 protected long killMasterTimeout;
78 protected long startMasterTimeout;
79 protected long killRsTimeout;
80 protected long startRsTimeout;
81 protected long killZkNodeTimeout;
82 protected long startZkNodeTimeout;
83 protected long killDataNodeTimeout;
84 protected long startDataNodeTimeout;
85
86 public void init(ActionContext context) throws IOException {
87 this.context = context;
88 cluster = context.getHBaseCluster();
89 initialStatus = cluster.getInitialClusterStatus();
90 Collection<ServerName> regionServers = initialStatus.getServers();
91 initialServers = regionServers.toArray(new ServerName[regionServers.size()]);
92
93 killMasterTimeout = cluster.getConf().getLong(KILL_MASTER_TIMEOUT_KEY,
94 KILL_MASTER_TIMEOUT_DEFAULT);
95 startMasterTimeout = cluster.getConf().getLong(START_MASTER_TIMEOUT_KEY,
96 START_MASTER_TIMEOUT_DEFAULT);
97 killRsTimeout = cluster.getConf().getLong(KILL_RS_TIMEOUT_KEY, KILL_RS_TIMEOUT_DEFAULT);
98 startRsTimeout = cluster.getConf().getLong(START_RS_TIMEOUT_KEY, START_RS_TIMEOUT_DEFAULT);
99 killZkNodeTimeout = cluster.getConf().getLong(KILL_ZK_NODE_TIMEOUT_KEY,
100 KILL_ZK_NODE_TIMEOUT_DEFAULT);
101 startZkNodeTimeout = cluster.getConf().getLong(START_ZK_NODE_TIMEOUT_KEY,
102 START_ZK_NODE_TIMEOUT_DEFAULT);
103 killDataNodeTimeout = cluster.getConf().getLong(KILL_DATANODE_TIMEOUT_KEY,
104 KILL_DATANODE_TIMEOUT_DEFAULT);
105 startDataNodeTimeout = cluster.getConf().getLong(START_DATANODE_TIMEOUT_KEY,
106 START_DATANODE_TIMEOUT_DEFAULT);
107 }
108
109 public void perform() throws Exception { }
110
111
112 protected ServerName[] getCurrentServers() throws IOException {
113 ClusterStatus clusterStatus = cluster.getClusterStatus();
114 Collection<ServerName> regionServers = clusterStatus.getServers();
115 int count = regionServers == null ? 0 : regionServers.size();
116 if (count <= 0) {
117 return new ServerName [] {};
118 }
119 ServerName master = clusterStatus.getMaster();
120 if (master == null || !regionServers.contains(master)) {
121 return regionServers.toArray(new ServerName[count]);
122 }
123 if (count == 1) {
124 return new ServerName [] {};
125 }
126 ArrayList<ServerName> tmp = new ArrayList<ServerName>(count);
127 tmp.addAll(regionServers);
128 tmp.remove(master);
129 return tmp.toArray(new ServerName[count-1]);
130 }
131
132 protected void killMaster(ServerName server) throws IOException {
133 LOG.info("Killing master:" + server);
134 cluster.killMaster(server);
135 cluster.waitForMasterToStop(server, killMasterTimeout);
136 LOG.info("Killed master server:" + server);
137 }
138
139 protected void startMaster(ServerName server) throws IOException {
140 LOG.info("Starting master:" + server.getHostname());
141 cluster.startMaster(server.getHostname(), server.getPort());
142 cluster.waitForActiveAndReadyMaster(startMasterTimeout);
143 LOG.info("Started master: " + server);
144 }
145
146 protected void killRs(ServerName server) throws IOException {
147 LOG.info("Killing region server:" + server);
148 cluster.killRegionServer(server);
149 cluster.waitForRegionServerToStop(server, killRsTimeout);
150 LOG.info("Killed region server:" + server + ". Reported num of rs:"
151 + cluster.getClusterStatus().getServersSize());
152 }
153
154 protected void startRs(ServerName server) throws IOException {
155 LOG.info("Starting region server:" + server.getHostname());
156 cluster.startRegionServer(server.getHostname(), server.getPort());
157 cluster.waitForRegionServerToStart(server.getHostname(), server.getPort(), startRsTimeout);
158 LOG.info("Started region server:" + server + ". Reported num of rs:"
159 + cluster.getClusterStatus().getServersSize());
160 }
161
162 protected void killZKNode(ServerName server) throws IOException {
163 LOG.info("Killing zookeeper node:" + server);
164 cluster.killZkNode(server);
165 cluster.waitForZkNodeToStop(server, killZkNodeTimeout);
166 LOG.info("Killed zookeeper node:" + server + ". Reported num of rs:"
167 + cluster.getClusterStatus().getServersSize());
168 }
169
170 protected void startZKNode(ServerName server) throws IOException {
171 LOG.info("Starting zookeeper node:" + server.getHostname());
172 cluster.startZkNode(server.getHostname(), server.getPort());
173 cluster.waitForZkNodeToStart(server, startZkNodeTimeout);
174 LOG.info("Started zookeeper node:" + server);
175 }
176
177 protected void killDataNode(ServerName server) throws IOException {
178 LOG.info("Killing datanode:" + server);
179 cluster.killDataNode(server);
180 cluster.waitForDataNodeToStop(server, killDataNodeTimeout);
181 LOG.info("Killed datanode:" + server + ". Reported num of rs:"
182 + cluster.getClusterStatus().getServersSize());
183 }
184
185 protected void startDataNode(ServerName server) throws IOException {
186 LOG.info("Starting datanode:" + server.getHostname());
187 cluster.startDataNode(server);
188 cluster.waitForDataNodeToStart(server, startDataNodeTimeout);
189 LOG.info("Started datanode:" + server);
190 }
191
192 protected void unbalanceRegions(ClusterStatus clusterStatus,
193 List<ServerName> fromServers, List<ServerName> toServers,
194 double fractionOfRegions) throws Exception {
195 List<byte[]> victimRegions = new LinkedList<byte[]>();
196 for (ServerName server : fromServers) {
197 ServerLoad serverLoad = clusterStatus.getLoad(server);
198
199 List<byte[]> regions = new LinkedList<byte[]>(serverLoad.getRegionsLoad().keySet());
200 int victimRegionCount = (int)Math.ceil(fractionOfRegions * regions.size());
201 LOG.debug("Removing " + victimRegionCount + " regions from " + server.getServerName());
202 for (int i = 0; i < victimRegionCount; ++i) {
203 int victimIx = RandomUtils.nextInt(regions.size());
204 String regionId = HRegionInfo.encodeRegionName(regions.remove(victimIx));
205 victimRegions.add(Bytes.toBytes(regionId));
206 }
207 }
208
209 LOG.info("Moving " + victimRegions.size() + " regions from " + fromServers.size()
210 + " servers to " + toServers.size() + " different servers");
211 Admin admin = this.context.getHBaseIntegrationTestingUtility().getHBaseAdmin();
212 for (byte[] victimRegion : victimRegions) {
213
214
215 if (context.isStopping()) {
216 break;
217 }
218 int targetIx = RandomUtils.nextInt(toServers.size());
219 admin.move(victimRegion, Bytes.toBytes(toServers.get(targetIx).getServerName()));
220 }
221 }
222
223 protected void forceBalancer() throws Exception {
224 Admin admin = this.context.getHBaseIntegrationTestingUtility().getHBaseAdmin();
225 boolean result = false;
226 try {
227 result = admin.balancer();
228 } catch (Exception e) {
229 LOG.warn("Got exception while doing balance ", e);
230 }
231 if (!result) {
232 LOG.error("Balancer didn't succeed");
233 }
234 }
235
236 public Configuration getConf() {
237 return cluster.getConf();
238 }
239
240
241
242
243 public static class ActionContext {
244 private IntegrationTestingUtility util;
245
246 public ActionContext(IntegrationTestingUtility util) {
247 this.util = util;
248 }
249
250 public IntegrationTestingUtility getHBaseIntegrationTestingUtility() {
251 return util;
252 }
253
254 public HBaseCluster getHBaseCluster() {
255 return util.getHBaseClusterInterface();
256 }
257
258 public boolean isStopping() {
259 return false;
260 }
261 }
262 }