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.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   * The {@link RpcServerEngine} implementation for ProtoBuf-based RPCs.
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      /** Default value for above params */
76      private static final int DEFAULT_WARN_RESPONSE_TIME = 10000; // milliseconds
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       // continue with base startup
138       super.startThreads();
139     }
140 
141     @Override
142     /**
143      * This is a server side method, which is invoked over RPC. On success
144      * the return response has protobuf response payload. On failure, the
145      * exception name and the stack trace are returned in the protobuf response.
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         // TODO: Review after we add in encoded data blocks.
158         status.setRPCPacket(param);
159         status.resume("Servicing call");
160         //get an instance of the method arg type
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           // Always create a controller.  Some invocations may not pass data in but will pass
172           // data out and they'll need a controller instance to carry it for them.
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         // log any RPC responses that are slower than the configured warn
195         // response time or larger than configured warning size
196         boolean tooSlow = (processingTime > warnResponseTime && warnResponseTime > -1);
197         boolean tooLarge = (responseSize > warnResponseSize && warnResponseSize > -1);
198         if (tooSlow || tooLarge) {
199           // when tagging, we let TooLarge trump TooSmall to keep output simple
200           // note that large responses will often also be slow.
201           // TOOD: This output is useless.... output the serialized pb as toString but do a
202           // short form, shorter than TextFormat.shortDebugString(proto).
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      * Logs an RPC response to the LOG file, producing valid JSON objects for
238      * client Operations.
239      * @param params The parameters received in the call.
240      * @param methodName The name of the method invoked
241      * @param call The string representation of the call
242      * @param tag  The tag that will be used to indicate this event in the log.
243      * @param clientAddress   The address of the client who made this call.
244      * @param startTime       The time that the call was initiated, in ms.
245      * @param processingTime  The duration that the call took to run, in ms.
246      * @param qTime           The duration that the call spent on the queue
247      *                        prior to being initiated, in ms.
248      * @param responseSize    The size in bytes of the response buffer.
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       // for JSON encoding
255       ObjectMapper mapper = new ObjectMapper();
256       // base information that is reported regardless of type of call
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         // if the slow process is a query, we want to log its table as well
269         // as its own fingerprint
270         byte [] tableName =
271             HRegionInfo.parseRegionName((byte[]) params[0])[0];
272         responseInfo.put("table", Bytes.toStringBinary(tableName));
273         // annotate the response map with operation details
274         responseInfo.putAll(((Operation) params[1]).toMap());
275         // report to the log file
276         LOG.warn("(operation" + tag + "): " +
277             mapper.writeValueAsString(responseInfo));
278       } else if (params.length == 1 && instance instanceof HRegionServer &&
279           params[0] instanceof Operation) {
280         // annotate the response map with operation details
281         responseInfo.putAll(((Operation) params[0]).toMap());
282         // report to the log file
283         LOG.warn("(operation" + tag + "): " +
284             mapper.writeValueAsString(responseInfo));
285       } else {
286         // can't get JSON details, so just report call.toString() along with
287         // a more generic tag.
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 }