1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase;
20
21 import java.io.IOException;
22
23 import org.apache.hadoop.hbase.classification.InterfaceAudience;
24 import org.apache.hadoop.hbase.util.Threads;
25 import org.apache.zookeeper.AsyncCallback;
26 import org.apache.zookeeper.CreateMode;
27 import org.apache.zookeeper.KeeperException;
28 import org.apache.zookeeper.WatchedEvent;
29 import org.apache.zookeeper.Watcher;
30 import org.apache.zookeeper.ZooDefs;
31 import org.apache.zookeeper.ZooKeeper;
32 import org.apache.zookeeper.data.Stat;
33 import org.slf4j.Logger;
34 import org.slf4j.LoggerFactory;
35
36 @InterfaceAudience.Private
37 public class ChaosZKClient {
38
39 private static final Logger LOG = LoggerFactory.getLogger(ChaosZKClient.class.getName());
40 private static final String CHAOS_AGENT_PARENT_ZNODE = "/hbase/chaosAgents";
41 private static final String CHAOS_AGENT_STATUS_ZNODE = "/hbase/chaosAgentTaskStatus";
42 private static final String ZNODE_PATH_SEPARATOR = "/";
43 private static final String TASK_PREFIX = "task_";
44 private static final String TASK_ERROR_STRING = "error";
45 private static final String TASK_COMPLETION_STRING = "done";
46 private static final String TASK_BOOLEAN_TRUE = "true";
47 private static final String TASK_BOOLEAN_FALSE = "false";
48 private static final String CONNECTION_LOSS = "ConnectionLoss";
49 private static final int SESSION_TIMEOUT_ZK = 10 * 60 * 1000;
50 private static final int TASK_EXECUTION_TIMEOUT = 5 * 60 * 1000;
51 private volatile String taskStatus = null;
52
53 private final String quorum;
54 private ZooKeeper zk;
55
56 public ChaosZKClient(String quorum) {
57 this.quorum = quorum;
58 try {
59 this.createNewZKConnection();
60 } catch (IOException e) {
61 LOG.error("Error creating ZooKeeper Connection: ", e);
62 }
63 }
64
65
66
67
68
69 public void createNewZKConnection() throws IOException {
70 Watcher watcher = new Watcher() {
71 @Override
72 public void process(WatchedEvent watchedEvent) {
73 LOG.info("Created ZooKeeper Connection For executing task");
74 }
75 };
76
77 this.zk = new ZooKeeper(quorum, SESSION_TIMEOUT_ZK, watcher);
78 }
79
80
81
82
83
84
85 private boolean isChaosAgentRunning(String hostname) {
86 try {
87 return zk.exists(CHAOS_AGENT_PARENT_ZNODE + ZNODE_PATH_SEPARATOR + hostname,
88 false) != null;
89 } catch (KeeperException e) {
90 if (e.toString().contains(CONNECTION_LOSS)) {
91 recreateZKConnection();
92 try {
93 return zk.exists(CHAOS_AGENT_PARENT_ZNODE + ZNODE_PATH_SEPARATOR + hostname,
94 false) != null;
95 } catch (KeeperException | InterruptedException ie) {
96 LOG.error("ERROR ", ie);
97 }
98 }
99 } catch (InterruptedException e) {
100 LOG.error("Error checking for given hostname: {} ERROR: ", hostname, e);
101 }
102 return false;
103 }
104
105
106
107
108
109
110
111 public String submitTask(final TaskObject taskObject) {
112 if (isChaosAgentRunning(taskObject.getTaskHostname())) {
113 LOG.info("Creating task node");
114 zk.create(CHAOS_AGENT_STATUS_ZNODE + ZNODE_PATH_SEPARATOR +
115 taskObject.getTaskHostname() + ZNODE_PATH_SEPARATOR + TASK_PREFIX,
116 taskObject.getCommand().getBytes(),
117 ZooDefs.Ids.OPEN_ACL_UNSAFE,
118 CreateMode.EPHEMERAL_SEQUENTIAL,
119 submitTaskCallback,
120 taskObject);
121 long start = System.currentTimeMillis();
122
123 while ((System.currentTimeMillis() - start) < TASK_EXECUTION_TIMEOUT) {
124 if(taskStatus != null) {
125 return taskStatus;
126 }
127 Threads.sleep(500);
128 }
129 } else {
130 LOG.info("EHHHHH! ChaosAgent Not running");
131 }
132 return TASK_ERROR_STRING;
133 }
134
135
136
137
138
139
140 private void getStatus(String path , Object ctx) {
141 LOG.info("Getting Status of task: " + path);
142 zk.getData(path,
143 false,
144 getStatusCallback,
145 ctx);
146 }
147
148
149
150
151
152
153 private void setStatusWatch(String name, TaskObject taskObject) {
154 LOG.info("Checking for ZNode and Setting watch for task : " + name);
155 zk.exists(name,
156 setStatusWatcher,
157 setStatusWatchCallback,
158 taskObject);
159 }
160
161
162
163
164
165 private void deleteTask(String path) {
166 LOG.info("Deleting task: " + path);
167 zk.delete(path,
168 -1,
169 taskDeleteCallback,
170 null);
171 }
172
173
174
175
176
177
178 Watcher setStatusWatcher = new Watcher() {
179 @Override
180 public void process(WatchedEvent watchedEvent) {
181 LOG.info("Setting status watch for task: " + watchedEvent.getPath());
182 if(watchedEvent.getType() == Event.EventType.NodeDataChanged) {
183 if(!watchedEvent.getPath().contains(TASK_PREFIX)) {
184 throw new RuntimeException(KeeperException.create(
185 KeeperException.Code.DATAINCONSISTENCY));
186 }
187 getStatus(watchedEvent.getPath(), (Object) watchedEvent.getPath());
188
189 }
190 }
191 };
192
193
194
195 AsyncCallback.DataCallback getStatusCallback = new AsyncCallback.DataCallback() {
196 @Override public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
197 switch (KeeperException.Code.get(rc)) {
198 case CONNECTIONLOSS:
199
200 ChaosZKClient.this.recreateZKConnection();
201 ChaosZKClient.this.getStatus(path, ctx);
202 break;
203
204 case OK:
205 if (ctx != null) {
206
207 String status = new String(data);
208 taskStatus = status;
209 switch (status) {
210 case TASK_COMPLETION_STRING:
211 case TASK_BOOLEAN_TRUE:
212 case TASK_BOOLEAN_FALSE:
213 LOG.info("Task executed completely : Status --> " + status);
214 break;
215
216 case TASK_ERROR_STRING:
217 LOG.info("There was error while executing task : Status --> " + status);
218 break;
219
220 default:
221 LOG.warn("Status of task is undefined!! : Status --> " + status);
222 }
223
224 ChaosZKClient.this.deleteTask(path);
225 }
226 break;
227
228 default:
229 LOG.error("ERROR while getting status of task: " + path + " ERROR: " + KeeperException
230 .create(KeeperException.Code.get(rc)));
231 }
232 }
233 };
234
235 AsyncCallback.StatCallback setStatusWatchCallback = new AsyncCallback.StatCallback() {
236 @Override public void processResult(int rc, String path, Object ctx, Stat stat) {
237 switch (KeeperException.Code.get(rc)) {
238 case CONNECTIONLOSS:
239
240 ChaosZKClient.this.recreateZKConnection();
241 ChaosZKClient.this.setStatusWatch(path, (TaskObject) ctx);
242 break;
243
244 case OK:
245 if (stat != null) {
246 ChaosZKClient.this.getStatus(path, null);
247 }
248 break;
249
250 default:
251 LOG.error(
252 "ERROR while setting watch on task ZNode: " + path + " ERROR: " + KeeperException
253 .create(KeeperException.Code.get(rc)));
254 }
255 }
256 };
257
258 AsyncCallback.StringCallback submitTaskCallback = new AsyncCallback.StringCallback() {
259 @Override public void processResult(int rc, String path, Object ctx, String name) {
260 switch (KeeperException.Code.get(rc)) {
261 case CONNECTIONLOSS:
262
263 ChaosZKClient.this.recreateZKConnection();
264 ChaosZKClient.this.submitTask((TaskObject) ctx);
265 break;
266
267 case OK:
268 LOG.info("Task created : " + name);
269 ChaosZKClient.this.setStatusWatch(name, (TaskObject) ctx);
270 break;
271
272 default:
273 LOG.error("Error submitting task: " + name + " ERROR:" + KeeperException
274 .create(KeeperException.Code.get(rc)));
275 }
276 }
277 };
278
279 AsyncCallback.VoidCallback taskDeleteCallback = new AsyncCallback.VoidCallback() {
280 @Override
281 public void processResult(int rc, String path, Object ctx) {
282 switch (KeeperException.Code.get(rc)) {
283 case CONNECTIONLOSS:
284
285 recreateZKConnection();
286 deleteTask(path);
287 break;
288
289 case OK:
290 LOG.info("Task Deleted successfully!");
291 LOG.info("Closing ZooKeeper Connection");
292 try {
293 zk.close();
294 } catch (InterruptedException e) {
295 LOG.error("Error while closing ZooKeeper Connection.");
296 }
297 break;
298
299 default:
300 LOG.error("ERROR while deleting task: " + path + " ERROR: " +
301 KeeperException.create(KeeperException.Code.get(rc)));
302 }
303 }
304 };
305
306
307 private void recreateZKConnection() {
308 try {
309 zk.close();
310 } catch (InterruptedException e) {
311 LOG.error("Error closing ZK connection : ", e);
312 } finally {
313 try {
314 createNewZKConnection();
315 } catch (IOException e) {
316 LOG.error("Error creating new ZK COnnection for agent: ", e);
317 }
318 }
319 }
320
321 static class TaskObject {
322 private final String command;
323 private final String taskHostname;
324
325 public TaskObject(String command, String taskHostname) {
326 this.command = command;
327 this.taskHostname = taskHostname;
328 }
329
330 public String getCommand() {
331 return this.command;
332 }
333
334 public String getTaskHostname() {
335 return taskHostname;
336 }
337 }
338
339 }