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  
19  package org.apache.hadoop.hbase.chaos;
20  
21  import java.io.Closeable;
22  import java.io.File;
23  import java.io.IOException;
24  import java.util.ArrayList;
25  import java.util.List;
26  import java.util.Map;
27  import java.util.concurrent.atomic.AtomicBoolean;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.hbase.classification.InterfaceAudience;
30  import org.apache.hadoop.hbase.util.Pair;
31  import org.apache.hadoop.hbase.util.RetryCounter;
32  import org.apache.hadoop.hbase.util.RetryCounterFactory;
33  import org.apache.hadoop.util.Shell;
34  import org.apache.zookeeper.AsyncCallback;
35  import org.apache.zookeeper.CreateMode;
36  import org.apache.zookeeper.KeeperException;
37  import org.apache.zookeeper.WatchedEvent;
38  import org.apache.zookeeper.Watcher;
39  import org.apache.zookeeper.ZooDefs;
40  import org.apache.zookeeper.ZooKeeper;
41  import org.apache.zookeeper.data.Stat;
42  import org.slf4j.Logger;
43  import org.slf4j.LoggerFactory;
44  
45  /***
46   * An agent for executing destructive actions for ChaosMonkey.
47   * Uses ZooKeeper Watchers and LocalShell, to do the killing
48   * and getting status of service on targeted host without SSH.
49   * uses given ZNode Structure:
50   *  /perfChaosTest (root)
51   *              |
52   *              |
53   *              /chaosAgents (Used for registration has
54   *              hostname ephemeral nodes as children)
55   *              |
56   *              |
57   *              /chaosAgentTaskStatus (Used for task
58   *              Execution, has hostname persistent
59   *              nodes as child with tasks as their children)
60   *                          |
61   *                          |
62   *                          /hostname
63   *                                |
64   *                                |
65   *                                /task0000001 (command as data)
66   *                                (has two types of command :
67   *                                     1: starts with "exec"
68   *                                       for executing a destructive action.
69   *                                     2: starts with "bool" for getting
70   *                                       only status of service.
71   *
72   */
73  @InterfaceAudience.Private
74  public class ChaosAgent implements Watcher, Closeable, Runnable {
75  
76    private static final Logger LOG = LoggerFactory.getLogger(ChaosAgent.class);
77    static AtomicBoolean stopChaosAgent = new AtomicBoolean();
78    private ZooKeeper zk;
79    private String quorum;
80    private String agentName;
81    private Configuration conf;
82    private RetryCounterFactory retryCounterFactory;
83    private volatile boolean connected = false;
84  
85    public ChaosAgent(Configuration conf, String quorum, String agentName) {
86      initChaosAgent(conf, quorum, agentName);
87    }
88  
89    /***
90     * sets global params and initiates connection with ZooKeeper then does registration.
91     * @param conf initial configuration to use
92     * @param quorum ZK Quorum
93     * @param agentName AgentName to use
94     */
95    private void initChaosAgent(Configuration conf, String quorum, String agentName) {
96      this.conf = conf;
97      this.quorum = quorum;
98      this.agentName = agentName;
99      this.retryCounterFactory = new RetryCounterFactory(new RetryCounter.RetryConfig()
100       .setMaxAttempts(conf.getInt(ChaosConstants.RETRY_ATTEMPTS_KEY,
101         ChaosConstants.DEFAULT_RETRY_ATTEMPTS)).setSleepInterval(
102           conf.getLong(ChaosConstants.RETRY_SLEEP_INTERVAL_KEY,
103             ChaosConstants.DEFAULT_RETRY_SLEEP_INTERVAL)));
104     try {
105       this.createZKConnection(null);
106       this.register();
107     } catch (IOException e) {
108       LOG.error("Error Creating Connection: " + e);
109     }
110   }
111 
112   /***
113    * Creates Connection with ZooKeeper.
114    * @throws IOException if something goes wrong
115    */
116   private void createZKConnection(Watcher watcher) throws IOException {
117     if(watcher == null) {
118       zk = new ZooKeeper(quorum, ChaosConstants.SESSION_TIMEOUT_ZK, this);
119     } else {
120       zk = new ZooKeeper(quorum, ChaosConstants.SESSION_TIMEOUT_ZK, watcher);
121     }
122     LOG.info("ZooKeeper Connection created for ChaosAgent: " + agentName);
123   }
124 
125   //WATCHERS: Below are the Watches used by ChaosAgent
126 
127   /***
128    * Watcher for notifying if any task is assigned to agent or not,
129    * by seeking if any Node is being added to agent as Child.
130    */
131   Watcher newTaskCreatedWatcher = new Watcher() {
132     @Override
133     public void process(WatchedEvent watchedEvent) {
134       if (watchedEvent.getType() == Event.EventType.NodeChildrenChanged) {
135         if (!(ChaosConstants.CHAOS_AGENT_STATUS_PERSISTENT_ZNODE +
136           ChaosConstants.ZNODE_PATH_SEPARATOR + agentName).equals(watchedEvent.getPath())) {
137           throw new RuntimeException(KeeperException.create(
138             KeeperException.Code.DATAINCONSISTENCY));
139         }
140 
141         LOG.info("Change in Tasks Node, checking for Tasks again.");
142         getTasks();
143       }
144 
145     }
146   };
147 
148   //CALLBACKS: Below are the Callbacks used by Chaos Agent
149 
150   /**
151    * Callback used while setting status of a given task, Logs given status.
152    */
153   AsyncCallback.StatCallback setStatusOfTaskZNodeCallback = new AsyncCallback.StatCallback() {
154     @Override public void processResult(int rc, String path, Object ctx, Stat stat) {
155       switch (KeeperException.Code.get(rc)) {
156         case CONNECTIONLOSS:
157           // Connection to the server was lost while setting status setting again.
158           try {
159             ChaosAgent.this.recreateZKConnection();
160           } catch (Exception e) {
161             break;
162           }
163           ChaosAgent.this.setStatusOfTaskZNode(path, (String) ctx);
164           break;
165 
166         case OK:
167           LOG.info("Status of Task has been set");
168           break;
169 
170         case NONODE:
171           LOG.error("Chaos Agent status node does not exists: "
172             + "check for ZNode directory structure again.");
173           break;
174 
175         default:
176           LOG.error("Error while setting status of task ZNode: " + path,
177             KeeperException.create(KeeperException.Code.get(rc), path));
178       }
179     }
180   };
181 
182   /**
183    * Callback used while creating a Persistent ZNode tries to create
184    * ZNode again if Connection was lost in previous try.
185    */
186   AsyncCallback.StringCallback createZNodeCallback = new AsyncCallback.StringCallback() {
187     @Override public void processResult(int rc, String path, Object ctx, String name) {
188       switch (KeeperException.Code.get(rc)) {
189         case CONNECTIONLOSS:
190           try {
191             ChaosAgent.this.recreateZKConnection();
192           } catch (Exception e) {
193             break;
194           }
195           ChaosAgent.this.createZNode(path, (byte[]) ctx);
196           break;
197         case OK:
198           LOG.info("ZNode created : " + path);
199           break;
200         case NODEEXISTS:
201           LOG.warn("ZNode already registered: " + path);
202           break;
203         default:
204           LOG.error("Error occurred while creating Persistent ZNode: " + path,
205             KeeperException.create(KeeperException.Code.get(rc), path));
206       }
207     }
208   };
209 
210   /**
211    * Callback used while creating a Ephemeral ZNode tries to create ZNode again
212    * if Connection was lost in previous try.
213    */
214   AsyncCallback.StringCallback createEphemeralZNodeCallback = new AsyncCallback.StringCallback() {
215     @Override public void processResult(int rc, String path, Object ctx, String name) {
216       switch (KeeperException.Code.get(rc)) {
217         case CONNECTIONLOSS:
218           try {
219             ChaosAgent.this.recreateZKConnection();
220           } catch (Exception e) {
221             break;
222           }
223           ChaosAgent.this.createEphemeralZNode(path, (byte[]) ctx);
224           break;
225         case OK:
226           LOG.info("ZNode created : " + path);
227           break;
228         case NODEEXISTS:
229           LOG.warn("ZNode already registered: " + path);
230           break;
231         default:
232           LOG.error("Error occurred while creating Ephemeral ZNode: ",
233             KeeperException.create(KeeperException.Code.get(rc), path));
234       }
235     }
236   };
237 
238   /**
239    * Callback used by getTasksForAgentCallback while getting command,
240    * after getting command successfully, it executes command and
241    * set its status with respect to the command type.
242    */
243   AsyncCallback.DataCallback getTaskForExecutionCallback = new AsyncCallback.DataCallback() {
244     @Override
245     public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
246       switch (KeeperException.Code.get(rc)) {
247         case CONNECTIONLOSS:
248           //Connection to the server has been lost while getting task, getting data again.
249           try {
250             recreateZKConnection();
251           } catch (Exception e) {
252             break;
253           }
254           zk.getData(path,
255             false,
256             getTaskForExecutionCallback,
257             new String(data));
258           break;
259         case OK:
260           String cmd = new String(data);
261           LOG.info("Executing command : " + cmd);
262           String status = ChaosConstants.TASK_COMPLETION_STRING;
263           try {
264             String user = conf.get(ChaosConstants.CHAOSAGENT_SHELL_USER,
265               ChaosConstants.DEFAULT_SHELL_USER);
266             switch (cmd.substring(0, 4)) {
267               case "bool":
268                 String ret = execWithRetries(user, cmd.substring(4)).getSecond();
269                 status = Boolean.toString(ret.length() > 0);
270                 break;
271 
272               case "exec":
273                 execWithRetries(user, cmd.substring(4));
274                 break;
275 
276               default:
277                 LOG.error("Unknown Command Type");
278                 status = ChaosConstants.TASK_ERROR_STRING;
279             }
280           } catch (IOException e) {
281             LOG.error("Got error while executing command : " + cmd +
282               " On agent : " + agentName + " Error : " + e);
283             status = ChaosConstants.TASK_ERROR_STRING;
284           }
285 
286           try {
287             setStatusOfTaskZNode(path, status);
288             Thread.sleep(ChaosConstants.SET_STATUS_SLEEP_TIME);
289           } catch (InterruptedException e) {
290             LOG.error("Error occured after setting status: " + e);
291           }
292 
293         default:
294           LOG.error("Error occurred while getting data",
295             KeeperException.create(KeeperException.Code.get(rc), path));
296       }
297     }
298   };
299 
300   /***
301    * Callback used while getting Tasks for agent if call executed without Exception,
302    * It creates a separate thread for each children to execute given Tasks parallely.
303    */
304   AsyncCallback.ChildrenCallback getTasksForAgentCallback = new AsyncCallback.ChildrenCallback() {
305     @Override
306     public void processResult(int rc, String path, Object ctx, List<String> children) {
307       switch (KeeperException.Code.get(rc)) {
308         case CONNECTIONLOSS: {
309           // Connection to the server has been lost, getting tasks again.
310           try {
311             recreateZKConnection();
312           } catch (Exception e) {
313             break;
314           }
315           getTasks();
316           break;
317         }
318 
319         case OK: {
320           if (children != null) {
321             try {
322 
323               LOG.info("Executing each task as a separate thread");
324               List<Thread> tasksList = new ArrayList<>();
325               for (final String task : children) {
326                 String threadName = agentName + "_" + task;
327                 Thread t = new Thread(new Runnable() {
328                   @Override
329                   public void run() {
330                     LOG.info("Executing task : " + task + " of agent : " + agentName);
331                     zk.getData(ChaosConstants.CHAOS_AGENT_STATUS_PERSISTENT_ZNODE
332                         + ChaosConstants.ZNODE_PATH_SEPARATOR + agentName
333                         + ChaosConstants.ZNODE_PATH_SEPARATOR + task, false,
334                       getTaskForExecutionCallback, task);
335 
336                   }
337                 });
338                 t.setName(threadName);
339                 t.start();
340                 tasksList.add(t);
341 
342                 for (Thread thread : tasksList) {
343                   thread.join();
344                 }
345               }
346             } catch (InterruptedException e) {
347               LOG.error("Error scheduling next task : " +
348                 " for agent : " + agentName + " Error : " + e);
349             }
350           }
351           break;
352         }
353 
354         default:
355           LOG.error("Error occurred while getting task",
356             KeeperException.create(KeeperException.Code.get(rc), path));
357       }
358     }
359   };
360 
361   /***
362    * Function to create PERSISTENT ZNODE with given path and data given as params
363    * @param path Path at which ZNode to create
364    * @param data Data to put under ZNode
365    */
366   public void createZNode(String path, byte[] data) {
367     zk.create(path,
368       data,
369       ZooDefs.Ids.OPEN_ACL_UNSAFE,
370       CreateMode.PERSISTENT,
371       createZNodeCallback,
372       data);
373   }
374 
375   /***
376    * Function to create EPHEMERAL ZNODE with given path and data as params.
377    * @param path Path at which Ephemeral ZNode to create
378    * @param data Data to put under ZNode
379    */
380   public void createEphemeralZNode(String path, byte[] data) {
381     zk.create(path,
382       data,
383       ZooDefs.Ids.OPEN_ACL_UNSAFE,
384       CreateMode.EPHEMERAL,
385       createEphemeralZNodeCallback,
386       data);
387   }
388 
389   /**
390    * Checks if given ZNode exists, if not creates a PERSISTENT ZNODE for same.
391    *
392    * @param path Path to check for ZNode
393    */
394   private void createIfZNodeNotExists(String path) {
395     try {
396       if (zk.exists(path,
397         false) == null) {
398         createZNode(path, new byte[0]);
399       }
400     } catch (KeeperException | InterruptedException e) {
401       LOG.error("Error checking given node : " + path + " " + e);
402     }
403   }
404 
405   /**
406    * sets given Status for Task Znode
407    *
408    * @param taskZNode ZNode to set status
409    * @param status Status value
410    */
411   public void setStatusOfTaskZNode(String taskZNode, String status) {
412     LOG.info("Setting status of Task ZNode: " + taskZNode + " status : " + status);
413     zk.setData(taskZNode,
414       status.getBytes(),
415       -1,
416       setStatusOfTaskZNodeCallback,
417       null);
418   }
419 
420   /**
421    * registration of ChaosAgent by checking and creating necessary ZNodes.
422    */
423   private void register() {
424     createIfZNodeNotExists(ChaosConstants.CHAOS_TEST_ROOT_ZNODE);
425     createIfZNodeNotExists(ChaosConstants.CHAOS_AGENT_REGISTRATION_EPIMERAL_ZNODE);
426     createIfZNodeNotExists(ChaosConstants.CHAOS_AGENT_STATUS_PERSISTENT_ZNODE);
427     createIfZNodeNotExists(ChaosConstants.CHAOS_AGENT_STATUS_PERSISTENT_ZNODE +
428       ChaosConstants.ZNODE_PATH_SEPARATOR + agentName);
429 
430     createEphemeralZNode(ChaosConstants.CHAOS_AGENT_REGISTRATION_EPIMERAL_ZNODE +
431       ChaosConstants.ZNODE_PATH_SEPARATOR + agentName, new byte[0]);
432   }
433 
434   /***
435    * Gets tasks for execution, basically sets Watch on it's respective host's Znode and
436    * waits for tasks to be assigned, also has a getTasksForAgentCallback
437    * which handles execution of task.
438    */
439   private void getTasks() {
440     LOG.info("Getting Tasks for Agent: " + agentName + "and setting watch for new Tasks");
441     zk.getChildren(ChaosConstants.CHAOS_AGENT_STATUS_PERSISTENT_ZNODE +
442         ChaosConstants.ZNODE_PATH_SEPARATOR + agentName,
443       newTaskCreatedWatcher,
444       getTasksForAgentCallback,
445       null);
446   }
447 
448   /**
449    * Below function executes command with retries with given user.
450    * Uses LocalShell to execute a command.
451    *
452    * @param user user name, default none
453    * @param cmd Command to execute
454    * @return A pair of Exit Code and Shell output
455    * @throws IOException Exception while executing shell command
456    */
457   private Pair<Integer, String> execWithRetries(String user, String cmd) throws IOException {
458     RetryCounter retryCounter = retryCounterFactory.create();
459     while (true) {
460       try {
461         return exec(user, cmd);
462       } catch (IOException e) {
463         retryOrThrow(retryCounter, e, user, cmd);
464       }
465       try {
466         retryCounter.sleepUntilNextRetry();
467       } catch (InterruptedException e) {
468         LOG.warn("Sleep Interrupted: " + e);
469       }
470     }
471   }
472 
473   private Pair<Integer, String> exec(String user, String cmd) throws IOException {
474     LOG.info("Executing Shell command: " + cmd + " , user: " + user);
475 
476     LocalShell shell = new LocalShell(user, cmd);
477     try {
478       shell.execute();
479     } catch (Shell.ExitCodeException e) {
480       String output = shell.getOutput();
481       throw new Shell.ExitCodeException(e.getExitCode(), "stderr: " + e.getMessage()
482         + ", stdout: " + output);
483     }
484     LOG.info("Executed Shell command, exit code: {}, output n{}", shell.getExitCode(), shell.getOutput());
485 
486     return new Pair<>(shell.getExitCode(), shell.getOutput());
487   }
488 
489   private <E extends Exception> void retryOrThrow(RetryCounter retryCounter, E ex,
490     String user, String cmd) throws E {
491     if (retryCounter.shouldRetry()) {
492       LOG.warn("Local command: {}, user: {}, failed at attempt {}. Retrying until maxAttempts: {}."
493         + "Exception {}", cmd, user,retryCounter.getAttemptTimes(), retryCounter.getMaxAttempts(),
494         ex.getMessage());
495       return;
496     }
497     throw ex;
498   }
499 
500   private boolean isConnected() {
501     return connected;
502   }
503 
504   @Override
505   public void close() throws IOException {
506     LOG.info("Closing ZooKeeper Connection for Chaos Agent : " + agentName);
507     try {
508       zk.close();
509     } catch (InterruptedException e) {
510       LOG.error("Error while closing ZooKeeper Connection.");
511     }
512   }
513 
514   @Override
515   public void run() {
516     try {
517       LOG.info("Running Chaos Agent on : " + agentName);
518       while (!this.isConnected()) {
519         Thread.sleep(100);
520       }
521       this.getTasks();
522       while (!stopChaosAgent.get()) {
523         Thread.sleep(500);
524       }
525     } catch (InterruptedException e) {
526       LOG.error("Error while running Chaos Agent", e);
527     }
528 
529   }
530 
531   @Override
532   public void process(WatchedEvent watchedEvent) {
533     LOG.info("Processing event: " + watchedEvent.toString());
534     if (watchedEvent.getType() == Event.EventType.None) {
535       switch (watchedEvent.getState()) {
536         case SyncConnected:
537           connected = true;
538           break;
539         case Disconnected:
540           connected = false;
541           break;
542         case Expired:
543           connected = false;
544           LOG.error("Session expired creating again");
545           try {
546             createZKConnection(null);
547           } catch (IOException e) {
548             LOG.error("Error creating Zookeeper connection", e);
549           }
550         default:
551           LOG.error("Unknown State");
552           break;
553       }
554     }
555   }
556 
557   private void recreateZKConnection() throws Exception{
558     try {
559       zk.close();
560         createZKConnection(newTaskCreatedWatcher);
561         createEphemeralZNode(ChaosConstants.CHAOS_AGENT_REGISTRATION_EPIMERAL_ZNODE +
562           ChaosConstants.ZNODE_PATH_SEPARATOR + agentName, new byte[0]);
563       } catch (IOException e) {
564         LOG.error("Error creating new ZK COnnection for agent: {}", agentName + e);
565         throw e;
566       }
567     }
568 
569   /**
570    * Executes Command locally.
571    */
572   protected static class LocalShell extends Shell.ShellCommandExecutor {
573 
574     private String user;
575     private String execCommand;
576 
577     public LocalShell(String user, String execCommand) {
578       super(new String[]{execCommand});
579       this.user = user;
580       this.execCommand = execCommand;
581     }
582 
583     @Override
584     public String[] getExecString() {
585       // TODO: Considering Agent is running with same user.
586       if(!user.equals(ChaosConstants.DEFAULT_SHELL_USER)){
587         execCommand = String.format("su -u %1$s %2$s", user, execCommand);
588       }
589       return new String[]{"/usr/bin/env", "bash", "-c", execCommand};
590     }
591 
592     @Override
593     public void execute() throws IOException {
594       super.execute();
595     }
596   }
597 }