1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.ipc;
20
21 import java.io.IOException;
22 import java.lang.reflect.InvocationTargetException;
23 import java.lang.reflect.Method;
24 import java.util.HashMap;
25 import java.util.Map;
26
27 import org.apache.commons.logging.Log;
28 import org.apache.hadoop.classification.InterfaceAudience;
29 import org.apache.hadoop.conf.Configuration;
30 import org.apache.hadoop.hbase.CellScanner;
31 import org.apache.hadoop.hbase.HRegionInfo;
32 import org.apache.hadoop.hbase.IpcProtocol;
33 import org.apache.hadoop.hbase.client.Operation;
34 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
35 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
36 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
37 import org.apache.hadoop.hbase.regionserver.HRegionServer;
38 import org.apache.hadoop.hbase.security.HBasePolicyProvider;
39 import org.apache.hadoop.hbase.security.token.AuthenticationTokenSecretManager;
40 import org.apache.hadoop.hbase.util.Bytes;
41 import org.apache.hadoop.hbase.util.Pair;
42 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
43 import org.codehaus.jackson.map.ObjectMapper;
44
45 import com.google.protobuf.Message;
46 import com.google.protobuf.ServiceException;
47 import com.google.protobuf.TextFormat;
48
49
50
51 @InterfaceAudience.Private
52 class ProtobufRpcServerEngine implements RpcServerEngine {
53 ProtobufRpcServerEngine() {
54 super();
55 }
56
57 @Override
58 public Server getServer(Object instance, Class<?>[] ifaces,
59 String bindAddress, int port, int numHandlers, int metaHandlerCount,
60 boolean verbose, Configuration conf, int highPriorityLevel)
61 throws IOException {
62 return new Server(instance, ifaces, conf, bindAddress, port, numHandlers,
63 metaHandlerCount, verbose, highPriorityLevel);
64 }
65
66 public static class Server extends HBaseServer {
67 boolean verbose;
68 Object instance;
69 Class<?> implementation;
70 private static final String WARN_RESPONSE_TIME =
71 "hbase.ipc.warn.response.time";
72 private static final String WARN_RESPONSE_SIZE =
73 "hbase.ipc.warn.response.size";
74
75
76 private static final int DEFAULT_WARN_RESPONSE_TIME = 10000;
77 private static final int DEFAULT_WARN_RESPONSE_SIZE = 100 * 1024 * 1024;
78
79 private final int warnResponseTime;
80 private final int warnResponseSize;
81
82 private static String classNameBase(String className) {
83 String[] names = className.split("\\.", -1);
84 if (names == null || names.length == 0) {
85 return className;
86 }
87 return names[names.length-1];
88 }
89
90 public Server(Object instance, final Class<?>[] ifaces,
91 Configuration conf, String bindAddress, int port,
92 int numHandlers, int metaHandlerCount, boolean verbose,
93 int highPriorityLevel)
94 throws IOException {
95 super(bindAddress, port, numHandlers, metaHandlerCount,
96 conf, classNameBase(instance.getClass().getName()),
97 highPriorityLevel);
98 this.instance = instance;
99 this.implementation = instance.getClass();
100 this.verbose = verbose;
101
102 this.warnResponseTime = conf.getInt(WARN_RESPONSE_TIME,
103 DEFAULT_WARN_RESPONSE_TIME);
104 this.warnResponseSize = conf.getInt(WARN_RESPONSE_SIZE,
105 DEFAULT_WARN_RESPONSE_SIZE);
106 this.verbose = verbose;
107 this.instance = instance;
108 this.implementation = instance.getClass();
109 }
110
111 private AuthenticationTokenSecretManager createSecretManager(){
112 if (!isSecurityEnabled ||
113 !(instance instanceof org.apache.hadoop.hbase.Server)) {
114 return null;
115 }
116 org.apache.hadoop.hbase.Server server =
117 (org.apache.hadoop.hbase.Server)instance;
118 Configuration conf = server.getConfiguration();
119 long keyUpdateInterval =
120 conf.getLong("hbase.auth.key.update.interval", 24*60*60*1000);
121 long maxAge =
122 conf.getLong("hbase.auth.token.max.lifetime", 7*24*60*60*1000);
123 return new AuthenticationTokenSecretManager(conf, server.getZooKeeper(),
124 server.getServerName().toString(), keyUpdateInterval, maxAge);
125 }
126
127 @Override
128 public void startThreads() {
129 AuthenticationTokenSecretManager mgr = createSecretManager();
130 if (mgr != null) {
131 setSecretManager(mgr);
132 mgr.start();
133 }
134 this.authManager = new ServiceAuthorizationManager();
135 HBasePolicyProvider.init(conf, authManager);
136
137
138 super.startThreads();
139 }
140
141 @Override
142
143
144
145
146
147 public Pair<Message, CellScanner> call(Class<? extends IpcProtocol> protocol,
148 Method method, Message param, CellScanner cellScanner, long receiveTime,
149 MonitoredRPCHandler status)
150 throws IOException {
151 try {
152 if (verbose) {
153 LOG.info("callId: " + CurCall.get().id + " protocol: " + protocol.getName() +
154 " method: " + method.getName());
155 }
156 status.setRPC(method.getName(), new Object[]{param}, receiveTime);
157
158 status.setRPCPacket(param);
159 status.resume("Servicing call");
160
161 Message result;
162 Object impl = null;
163 if (protocol.isAssignableFrom(this.implementation)) {
164 impl = this.instance;
165 } else {
166 throw new UnknownProtocolException(protocol);
167 }
168 PayloadCarryingRpcController controller = null;
169 long startTime = System.currentTimeMillis();
170 if (method.getParameterTypes().length == 2) {
171
172
173 controller = new PayloadCarryingRpcController(cellScanner);
174 result = (Message)method.invoke(impl, controller, param);
175 } else {
176 throw new ServiceException("Wrong number of parameters for method: [" +
177 method.getName() + "]" + ", wanted: 2, actual: " + method.getParameterTypes().length);
178 }
179 int processingTime = (int) (System.currentTimeMillis() - startTime);
180 int qTime = (int) (startTime-receiveTime);
181 if (LOG.isTraceEnabled()) {
182 LOG.trace(CurCall.get().toString() +
183 " response: " + TextFormat.shortDebugString(result) +
184 " served: " + protocol.getSimpleName() +
185 " queueTime: " + qTime +
186 " processingTime: " + processingTime);
187 }
188 metrics.dequeuedCall(qTime);
189 metrics.processedCall(processingTime);
190 if (verbose) {
191 log("Return " + TextFormat.shortDebugString(result), LOG);
192 }
193 long responseSize = result.getSerializedSize();
194
195
196 boolean tooSlow = (processingTime > warnResponseTime && warnResponseTime > -1);
197 boolean tooLarge = (responseSize > warnResponseSize && warnResponseSize > -1);
198 if (tooSlow || tooLarge) {
199
200
201
202
203 StringBuilder buffer = new StringBuilder(256);
204 buffer.append(method.getName());
205 buffer.append("(");
206 buffer.append(param.getClass().getName());
207 buffer.append(")");
208 logResponse(new Object[]{param},
209 method.getName(), buffer.toString(), (tooLarge ? "TooLarge" : "TooSlow"),
210 status.getClient(), startTime, processingTime, qTime,
211 responseSize);
212 }
213 return new Pair<Message, CellScanner>(result,
214 controller != null? controller.cellScanner(): null);
215 } catch (InvocationTargetException e) {
216 Throwable target = e.getTargetException();
217 if (target instanceof IOException) {
218 throw (IOException)target;
219 }
220 if (target instanceof ServiceException) {
221 throw ProtobufUtil.getRemoteException((ServiceException)target);
222 }
223 IOException ioe = new IOException(target.toString());
224 ioe.setStackTrace(target.getStackTrace());
225 throw ioe;
226 } catch (Throwable e) {
227 if (!(e instanceof IOException)) {
228 LOG.error("Unexpected throwable object ", e);
229 }
230 IOException ioe = new IOException(e.toString());
231 ioe.setStackTrace(e.getStackTrace());
232 throw ioe;
233 }
234 }
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250 void logResponse(Object[] params, String methodName, String call, String tag,
251 String clientAddress, long startTime, int processingTime, int qTime,
252 long responseSize)
253 throws IOException {
254
255 ObjectMapper mapper = new ObjectMapper();
256
257 Map<String, Object> responseInfo = new HashMap<String, Object>();
258 responseInfo.put("starttimems", startTime);
259 responseInfo.put("processingtimems", processingTime);
260 responseInfo.put("queuetimems", qTime);
261 responseInfo.put("responsesize", responseSize);
262 responseInfo.put("client", clientAddress);
263 responseInfo.put("class", instance.getClass().getSimpleName());
264 responseInfo.put("method", methodName);
265 if (params.length == 2 && instance instanceof HRegionServer &&
266 params[0] instanceof byte[] &&
267 params[1] instanceof Operation) {
268
269
270 byte [] tableName =
271 HRegionInfo.parseRegionName((byte[]) params[0])[0];
272 responseInfo.put("table", Bytes.toStringBinary(tableName));
273
274 responseInfo.putAll(((Operation) params[1]).toMap());
275
276 LOG.warn("(operation" + tag + "): " +
277 mapper.writeValueAsString(responseInfo));
278 } else if (params.length == 1 && instance instanceof HRegionServer &&
279 params[0] instanceof Operation) {
280
281 responseInfo.putAll(((Operation) params[0]).toMap());
282
283 LOG.warn("(operation" + tag + "): " +
284 mapper.writeValueAsString(responseInfo));
285 } else {
286
287
288 responseInfo.put("call", call);
289 LOG.warn("(response" + tag + "): " +
290 mapper.writeValueAsString(responseInfo));
291 }
292 }
293
294 protected static void log(String value, Log LOG) {
295 String v = value;
296 final int max = 100;
297 if (v != null && v.length() > max)
298 v = v.substring(0, max) + "...";
299 LOG.info(v);
300 }
301 }
302 }