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.mapreduce.util;
20 
21 import java.io.IOException;
22 import java.util.Arrays;
23 
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 
27 import org.apache.hadoop.classification.InterfaceAudience;
28 import org.apache.hadoop.classification.InterfaceStability;
29 import org.apache.hadoop.util.Shell.ExitCodeException;
30 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
31 
32 /**
33  * Process tree related operations
34  */
35 @InterfaceAudience.Private
36 @InterfaceStability.Unstable
37 public class ProcessTree {
38 
39   private static final Log LOG = LogFactory.getLog(ProcessTree.class);
40 
41   public static final long DEFAULT_SLEEPTIME_BEFORE_SIGKILL = 5000L;
42 
43   private static final int SIGQUIT = 3;
44   private static final int SIGTERM = 15;
45   private static final int SIGKILL = 9;
46 
47   private static final String SIGQUIT_STR = "SIGQUIT";
48   private static final String SIGTERM_STR = "SIGTERM";
49   private static final String SIGKILL_STR = "SIGKILL";
50 
51 
52   public static final boolean isSetsidAvailable = isSetsidSupported();
isSetsidSupported()53   private static boolean isSetsidSupported() {
54     ShellCommandExecutor shexec = null;
55     boolean setsidSupported = true;
56     try {
57       String[] args = {"setsid", "bash", "-c", "echo $$"};
58       shexec = new ShellCommandExecutor(args);
59       shexec.execute();
60     } catch (IOException ioe) {
61       LOG.warn("setsid is not available on this machine. So not using it.");
62       setsidSupported = false;
63     } finally { // handle the exit code
64       LOG.info("setsid exited with exit code " + shexec.getExitCode());
65     }
66     return setsidSupported;
67   }
68 
69   /**
70    * Destroy the process-tree.
71    * @param pid process id of the root process of the subtree of processes
72    *            to be killed
73    * @param sleeptimeBeforeSigkill The time to wait before sending SIGKILL
74    *                               after sending SIGTERM
75    * @param isProcessGroup pid is a process group leader or not
76    * @param inBackground Process is to be killed in the back ground with
77    *                     a separate thread
78    */
destroy(String pid, long sleeptimeBeforeSigkill, boolean isProcessGroup, boolean inBackground)79   public static void destroy(String pid, long sleeptimeBeforeSigkill,
80                              boolean isProcessGroup, boolean inBackground) {
81     if(isProcessGroup) {
82       destroyProcessGroup(pid, sleeptimeBeforeSigkill, inBackground);
83     }
84     else {
85       //TODO: Destroy all the processes in the subtree in this case also.
86       // For the time being, killing only the root process.
87       destroyProcess(pid, sleeptimeBeforeSigkill, inBackground);
88     }
89   }
90 
91   /** Destroy the process.
92    * @param pid Process id of to-be-killed-process
93    * @param sleeptimeBeforeSigkill The time to wait before sending SIGKILL
94    *                               after sending SIGTERM
95    * @param inBackground Process is to be killed in the back ground with
96    *                     a separate thread
97    */
destroyProcess(String pid, long sleeptimeBeforeSigkill, boolean inBackground)98   protected static void destroyProcess(String pid, long sleeptimeBeforeSigkill,
99                                     boolean inBackground) {
100     terminateProcess(pid);
101     sigKill(pid, false, sleeptimeBeforeSigkill, inBackground);
102   }
103 
104   /** Destroy the process group.
105    * @param pgrpId Process group id of to-be-killed-processes
106    * @param sleeptimeBeforeSigkill The time to wait before sending SIGKILL
107    *                               after sending SIGTERM
108    * @param inBackground Process group is to be killed in the back ground with
109    *                     a separate thread
110    */
destroyProcessGroup(String pgrpId, long sleeptimeBeforeSigkill, boolean inBackground)111   protected static void destroyProcessGroup(String pgrpId,
112                        long sleeptimeBeforeSigkill, boolean inBackground) {
113     terminateProcessGroup(pgrpId);
114     sigKill(pgrpId, true, sleeptimeBeforeSigkill, inBackground);
115   }
116 
117 
118   /**
119    * Send a specified signal to the specified pid
120    *
121    * @param pid the pid of the process [group] to signal.
122    * @param signalNum the signal to send.
123    * @param signalName the human-readable description of the signal
124    * (for logging).
125    */
sendSignal(String pid, int signalNum, String signalName)126   private static void sendSignal(String pid, int signalNum, String signalName) {
127     ShellCommandExecutor shexec = null;
128     try {
129       String[] args = { "kill", "-" + signalNum, pid };
130       shexec = new ShellCommandExecutor(args);
131       shexec.execute();
132     } catch (IOException ioe) {
133       LOG.warn("Error executing shell command " + ioe);
134     } finally {
135       if (pid.startsWith("-")) {
136         LOG.info("Sending signal to all members of process group " + pid
137             + ": " + signalName + ". Exit code " + shexec.getExitCode());
138       } else {
139         LOG.info("Signaling process " + pid
140             + " with " + signalName + ". Exit code " + shexec.getExitCode());
141       }
142     }
143   }
144 
145   /**
146    * Send a specified signal to the process, if it is alive.
147    *
148    * @param pid the pid of the process to signal.
149    * @param signalNum the signal to send.
150    * @param signalName the human-readable description of the signal
151    * (for logging).
152    * @param alwaysSignal if true then send signal even if isAlive(pid) is false
153    */
maybeSignalProcess(String pid, int signalNum, String signalName, boolean alwaysSignal)154   private static void maybeSignalProcess(String pid, int signalNum,
155       String signalName, boolean alwaysSignal) {
156     // If process tree is not alive then don't signal, unless alwaysSignal
157     // forces it so.
158     if (alwaysSignal || ProcessTree.isAlive(pid)) {
159       sendSignal(pid, signalNum, signalName);
160     }
161   }
162 
maybeSignalProcessGroup(String pgrpId, int signalNum, String signalName, boolean alwaysSignal)163   private static void maybeSignalProcessGroup(String pgrpId, int signalNum,
164       String signalName, boolean alwaysSignal) {
165 
166     if (alwaysSignal || ProcessTree.isProcessGroupAlive(pgrpId)) {
167       // signaling a process group means using a negative pid.
168       sendSignal("-" + pgrpId, signalNum, signalName);
169     }
170   }
171 
172   /**
173    * Sends terminate signal to the process, allowing it to gracefully exit.
174    *
175    * @param pid pid of the process to be sent SIGTERM
176    */
terminateProcess(String pid)177   public static void terminateProcess(String pid) {
178     maybeSignalProcess(pid, SIGTERM, SIGTERM_STR, true);
179   }
180 
181   /**
182    * Sends terminate signal to all the process belonging to the passed process
183    * group, allowing the group to gracefully exit.
184    *
185    * @param pgrpId process group id
186    */
terminateProcessGroup(String pgrpId)187   public static void terminateProcessGroup(String pgrpId) {
188     maybeSignalProcessGroup(pgrpId, SIGTERM, SIGTERM_STR, true);
189   }
190 
191   /**
192    * Kills the process(OR process group) by sending the signal SIGKILL
193    * in the current thread
194    * @param pid Process id(OR process group id) of to-be-deleted-process
195    * @param isProcessGroup Is pid a process group id of to-be-deleted-processes
196    * @param sleepTimeBeforeSigKill wait time before sending SIGKILL after
197    *  sending SIGTERM
198    */
sigKillInCurrentThread(String pid, boolean isProcessGroup, long sleepTimeBeforeSigKill)199   private static void sigKillInCurrentThread(String pid, boolean isProcessGroup,
200       long sleepTimeBeforeSigKill) {
201     // Kill the subprocesses of root process(even if the root process is not
202     // alive) if process group is to be killed.
203     if (isProcessGroup || ProcessTree.isAlive(pid)) {
204       try {
205         // Sleep for some time before sending SIGKILL
206         Thread.sleep(sleepTimeBeforeSigKill);
207       } catch (InterruptedException i) {
208         LOG.warn("Thread sleep is interrupted.");
209       }
210       if(isProcessGroup) {
211         killProcessGroup(pid);
212       } else {
213         killProcess(pid);
214       }
215     }
216   }
217 
218 
219   /** Kills the process(OR process group) by sending the signal SIGKILL
220    * @param pid Process id(OR process group id) of to-be-deleted-process
221    * @param isProcessGroup Is pid a process group id of to-be-deleted-processes
222    * @param sleeptimeBeforeSigkill The time to wait before sending SIGKILL
223    *                               after sending SIGTERM
224    * @param inBackground Process is to be killed in the back ground with
225    *                     a separate thread
226    */
sigKill(String pid, boolean isProcessGroup, long sleeptimeBeforeSigkill, boolean inBackground)227   private static void sigKill(String pid, boolean isProcessGroup,
228                         long sleeptimeBeforeSigkill, boolean inBackground) {
229 
230     if(inBackground) { // use a separate thread for killing
231       SigKillThread sigKillThread = new SigKillThread(pid, isProcessGroup,
232                                                       sleeptimeBeforeSigkill);
233       sigKillThread.setDaemon(true);
234       sigKillThread.start();
235     }
236     else {
237       sigKillInCurrentThread(pid, isProcessGroup, sleeptimeBeforeSigkill);
238     }
239   }
240 
241   /**
242    * Sends kill signal to process, forcefully terminating the process.
243    *
244    * @param pid process id
245    */
killProcess(String pid)246   public static void killProcess(String pid) {
247     maybeSignalProcess(pid, SIGKILL, SIGKILL_STR, false);
248   }
249 
250   /**
251    * Sends SIGQUIT to process; Java programs will dump their stack to
252    * stdout.
253    *
254    * @param pid process id
255    */
sigQuitProcess(String pid)256   public static void sigQuitProcess(String pid) {
257     maybeSignalProcess(pid, SIGQUIT, SIGQUIT_STR, false);
258   }
259 
260   /**
261    * Sends kill signal to all process belonging to same process group,
262    * forcefully terminating the process group.
263    *
264    * @param pgrpId process group id
265    */
killProcessGroup(String pgrpId)266   public static void killProcessGroup(String pgrpId) {
267     maybeSignalProcessGroup(pgrpId, SIGKILL, SIGKILL_STR, false);
268   }
269 
270   /**
271    * Sends SIGQUIT to all processes belonging to the same process group,
272    * ordering all processes in the group to send their stack dump to
273    * stdout.
274    *
275    * @param pgrpId process group id
276    */
sigQuitProcessGroup(String pgrpId)277   public static void sigQuitProcessGroup(String pgrpId) {
278     maybeSignalProcessGroup(pgrpId, SIGQUIT, SIGQUIT_STR, false);
279   }
280 
281   /**
282    * Is the process with PID pid still alive?
283    * This method assumes that isAlive is called on a pid that was alive not
284    * too long ago, and hence assumes no chance of pid-wrapping-around.
285    *
286    * @param pid pid of the process to check.
287    * @return true if process is alive.
288    */
isAlive(String pid)289   public static boolean isAlive(String pid) {
290     ShellCommandExecutor shexec = null;
291     try {
292       String[] args = { "kill", "-0", pid };
293       shexec = new ShellCommandExecutor(args);
294       shexec.execute();
295     } catch (ExitCodeException ee) {
296       return false;
297     } catch (IOException ioe) {
298       LOG.warn("Error executing shell command "
299           + shexec.toString() + ioe);
300       return false;
301     }
302     return (shexec.getExitCode() == 0 ? true : false);
303   }
304 
305   /**
306    * Is the process group with  still alive?
307    *
308    * This method assumes that isAlive is called on a pid that was alive not
309    * too long ago, and hence assumes no chance of pid-wrapping-around.
310    *
311    * @param pgrpId process group id
312    * @return true if any of process in group is alive.
313    */
isProcessGroupAlive(String pgrpId)314   public static boolean isProcessGroupAlive(String pgrpId) {
315     ShellCommandExecutor shexec = null;
316     try {
317       String[] args = { "kill", "-0", "-"+pgrpId };
318       shexec = new ShellCommandExecutor(args);
319       shexec.execute();
320     } catch (ExitCodeException ee) {
321       return false;
322     } catch (IOException ioe) {
323       LOG.warn("Error executing shell command "
324           + shexec.toString() + ioe);
325       return false;
326     }
327     return (shexec.getExitCode() == 0 ? true : false);
328   }
329 
330 
331   /**
332    * Helper thread class that kills process-tree with SIGKILL in background
333    */
334   static class SigKillThread extends Thread {
335     private String pid = null;
336     private boolean isProcessGroup = false;
337 
338     private long sleepTimeBeforeSigKill = DEFAULT_SLEEPTIME_BEFORE_SIGKILL;
339 
SigKillThread(String pid, boolean isProcessGroup, long interval)340     private SigKillThread(String pid, boolean isProcessGroup, long interval) {
341       this.pid = pid;
342       this.isProcessGroup = isProcessGroup;
343       this.setName(this.getClass().getName() + "-" + pid);
344       sleepTimeBeforeSigKill = interval;
345     }
346 
run()347     public void run() {
348       sigKillInCurrentThread(pid, isProcessGroup, sleepTimeBeforeSigKill);
349     }
350   }
351 }
352