1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase;
19
20 import com.google.common.util.concurrent.ThreadFactoryBuilder;
21 import com.google.protobuf.RpcCallback;
22 import com.google.protobuf.RpcController;
23 import com.google.protobuf.Service;
24 import java.io.IOException;
25 import java.util.concurrent.ExecutorService;
26 import java.util.concurrent.Executors;
27 import org.apache.commons.lang.StringUtils;
28 import org.apache.hadoop.conf.Configuration;
29 import org.apache.hadoop.hbase.chaos.monkies.ChaosMonkey;
30 import org.apache.hadoop.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
32 import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ShellExecEndpoint;
33 import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ShellExecEndpoint.ShellExecRequest;
34 import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ShellExecEndpoint.ShellExecResponse;
35 import org.apache.hadoop.util.Shell;
36 import org.slf4j.Logger;
37 import org.slf4j.LoggerFactory;
38
39
40
41
42
43 @InterfaceAudience.Private
44 public class ShellExecEndpointCoprocessor extends ShellExecEndpoint.ShellExecService implements
45 Coprocessor, CoprocessorService {
46 private static final Logger LOG = LoggerFactory.getLogger(ShellExecEndpointCoprocessor.class);
47
48 private static final String BACKGROUND_DELAY_MS_KEY = "hbase.it.shellexeccoproc.async.delay.ms";
49 private static final long DEFAULT_BACKGROUND_DELAY_MS = 1_000;
50
51 private final ExecutorService backgroundExecutor;
52 private Configuration conf;
53
54 public ShellExecEndpointCoprocessor() {
55 backgroundExecutor = Executors.newSingleThreadExecutor(
56 new ThreadFactoryBuilder()
57 .setNameFormat(ShellExecEndpointCoprocessor.class.getSimpleName() + "-{}")
58 .setDaemon(true)
59 .setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
60 @Override
61 public void uncaughtException(Thread t, Throwable e) {
62 LOG.warn(String.format("Thread %s threw %s", t, e));
63 }
64 }).build());
65 }
66
67 @Override
68 public void start(CoprocessorEnvironment env) {
69 conf = env.getConfiguration();
70 }
71
72 @Override
73 public void stop(CoprocessorEnvironment env) throws IOException {}
74
75 @Override
76 public void shellExec(
77 final RpcController controller,
78 final ShellExecRequest request,
79 final RpcCallback<ShellExecResponse> done
80 ) {
81 final String command = request.getCommand();
82 if (StringUtils.isBlank(command)) {
83 throw new RuntimeException("Request contained an empty command.");
84 }
85 final boolean awaitResponse = !request.hasAwaitResponse() || request.getAwaitResponse();
86 final String[] subShellCmd = new String[] { "/usr/bin/env", "bash", "-c", command };
87 final Shell.ShellCommandExecutor shell = new Shell.ShellCommandExecutor(subShellCmd);
88
89 final String msgFmt = "Executing command"
90 + (!awaitResponse ? " on a background thread" : "") + ": {}";
91 LOG.info(msgFmt, command);
92
93 if (awaitResponse) {
94 runForegroundTask(shell, controller, done);
95 } else {
96 runBackgroundTask(shell, done);
97 }
98 }
99
100 private void runForegroundTask(
101 final Shell.ShellCommandExecutor shell,
102 final RpcController controller,
103 final RpcCallback<ShellExecResponse> done
104 ) {
105 ShellExecResponse.Builder builder = ShellExecResponse.newBuilder();
106 try {
107 doExec(shell, builder);
108 } catch (IOException e) {
109 LOG.error("Failure launching process", e);
110 controller.setFailed(e.getMessage());
111 }
112 done.run(builder.build());
113 }
114
115 private void runBackgroundTask(
116 final Shell.ShellCommandExecutor shell,
117 final RpcCallback<ShellExecResponse> done
118 ) {
119 final long sleepDuration = conf.getLong(BACKGROUND_DELAY_MS_KEY, DEFAULT_BACKGROUND_DELAY_MS);
120 backgroundExecutor.submit(new Runnable() {
121 @Override
122 public void run() {
123 try {
124
125
126 Thread.sleep(sleepDuration);
127 ShellExecEndpointCoprocessor.this.doExec(shell, ShellExecResponse.newBuilder());
128 } catch (InterruptedException e) {
129 LOG.warn("Interrupted before launching process.", e);
130 } catch (IOException e) {
131 LOG.error("Failure launching process", e);
132 }
133 }
134 });
135 done.run(ShellExecResponse.newBuilder().build());
136 }
137
138
139
140
141 private void doExec(
142 final Shell.ShellCommandExecutor shell,
143 final ShellExecResponse.Builder builder
144 ) throws IOException {
145 try {
146 shell.execute();
147 builder
148 .setExitCode(shell.getExitCode())
149 .setStdout(shell.getOutput());
150 } catch (Shell.ExitCodeException e) {
151 LOG.warn("Launched process failed", e);
152 builder
153 .setExitCode(e.getExitCode())
154 .setStdout(shell.getOutput())
155 .setStderr(e.getMessage());
156 }
157 }
158
159 @Override
160 public Service getService() {
161 return this;
162 }
163 }