View Javadoc

1   /*
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  
21  package org.apache.hadoop.hbase.ipc;
22  
23  import java.lang.reflect.Proxy;
24  import java.lang.reflect.Method;
25  import java.lang.reflect.Array;
26  import java.lang.reflect.InvocationHandler;
27  import java.lang.reflect.InvocationTargetException;
28  
29  import java.net.InetSocketAddress;
30  import java.io.*;
31  import java.util.Map;
32  import java.util.HashMap;
33  
34  import javax.net.SocketFactory;
35  
36  import org.apache.commons.logging.*;
37  
38  import org.apache.hadoop.hbase.HRegionInfo;
39  import org.apache.hadoop.hbase.client.Operation;
40  import org.apache.hadoop.hbase.io.HbaseObjectWritable;
41  import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
42  import org.apache.hadoop.hbase.regionserver.HRegionServer;
43  import org.apache.hadoop.hbase.util.Bytes;
44  import org.apache.hadoop.hbase.util.Objects;
45  import org.apache.hadoop.io.*;
46  import org.apache.hadoop.ipc.RPC;
47  import org.apache.hadoop.hbase.ipc.VersionedProtocol;
48  import org.apache.hadoop.hbase.security.User;
49  import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
50  import org.apache.hadoop.conf.*;
51  
52  import org.codehaus.jackson.map.ObjectMapper;
53  
54  /** An RpcEngine implementation for Writable data. */
55  class WritableRpcEngine implements RpcEngine {
56    // LOG is NOT in hbase subpackage intentionally so that the default HBase
57    // DEBUG log level does NOT emit RPC-level logging. 
58    private static final Log LOG = LogFactory.getLog("org.apache.hadoop.ipc.RPCEngine");
59  
60    /* Cache a client using its socket factory as the hash key */
61    static private class ClientCache {
62      private Map<SocketFactory, HBaseClient> clients =
63        new HashMap<SocketFactory, HBaseClient>();
64  
65      protected ClientCache() {}
66  
67      /**
68       * Construct & cache an IPC client with the user-provided SocketFactory
69       * if no cached client exists.
70       *
71       * @param conf Configuration
72       * @param factory socket factory
73       * @return an IPC client
74       */
75      protected synchronized HBaseClient getClient(Configuration conf,
76          SocketFactory factory) {
77        // Construct & cache client.  The configuration is only used for timeout,
78        // and Clients have connection pools.  So we can either (a) lose some
79        // connection pooling and leak sockets, or (b) use the same timeout for
80        // all configurations.  Since the IPC is usually intended globally, not
81        // per-job, we choose (a).
82        HBaseClient client = clients.get(factory);
83        if (client == null) {
84          // Make an hbase client instead of hadoop Client.
85          client = new HBaseClient(HbaseObjectWritable.class, conf, factory);
86          clients.put(factory, client);
87        } else {
88          client.incCount();
89        }
90        return client;
91      }
92  
93      /**
94       * Construct & cache an IPC client with the default SocketFactory
95       * if no cached client exists.
96       *
97       * @param conf Configuration
98       * @return an IPC client
99       */
100     protected synchronized HBaseClient getClient(Configuration conf) {
101       return getClient(conf, SocketFactory.getDefault());
102     }
103 
104     /**
105      * Stop a RPC client connection
106      * A RPC client is closed only when its reference count becomes zero.
107      * @param client client to stop
108      */
109     protected void stopClient(HBaseClient client) {
110       synchronized (this) {
111         client.decCount();
112         if (client.isZeroReference()) {
113           clients.remove(client.getSocketFactory());
114         }
115       }
116       if (client.isZeroReference()) {
117         client.stop();
118       }
119     }
120   }
121 
122   protected final static ClientCache CLIENTS = new ClientCache();
123 
124   private static class Invoker implements InvocationHandler {
125     private Class<? extends VersionedProtocol> protocol;
126     private InetSocketAddress address;
127     private User ticket;
128     private HBaseClient client;
129     private boolean isClosed = false;
130     final private int rpcTimeout;
131 
132     public Invoker(Class<? extends VersionedProtocol> protocol,
133                    InetSocketAddress address, User ticket,
134                    Configuration conf, SocketFactory factory, int rpcTimeout) {
135       this.protocol = protocol;
136       this.address = address;
137       this.ticket = ticket;
138       this.client = CLIENTS.getClient(conf, factory);
139       this.rpcTimeout = rpcTimeout;
140     }
141 
142     public Object invoke(Object proxy, Method method, Object[] args)
143         throws Throwable {
144       final boolean logDebug = LOG.isDebugEnabled();
145       long startTime = 0;
146       if (logDebug) {
147         startTime = System.currentTimeMillis();
148       }
149 
150       HbaseObjectWritable value = (HbaseObjectWritable)
151         client.call(new Invocation(method, args), address,
152                     protocol, ticket, rpcTimeout);
153       if (logDebug) {
154         // FIGURE HOW TO TURN THIS OFF!
155         long callTime = System.currentTimeMillis() - startTime;
156         LOG.debug("Call: " + method.getName() + " " + callTime);
157       }
158       return value.get();
159     }
160 
161     /* close the IPC client that's responsible for this invoker's RPCs */
162     synchronized protected void close() {
163       if (!isClosed) {
164         isClosed = true;
165         CLIENTS.stopClient(client);
166       }
167     }
168   }
169 
170   /** Construct a client-side proxy object that implements the named protocol,
171    * talking to a server at the named address. */
172   public VersionedProtocol getProxy(
173       Class<? extends VersionedProtocol> protocol, long clientVersion,
174       InetSocketAddress addr, User ticket,
175       Configuration conf, SocketFactory factory, int rpcTimeout)
176     throws IOException {
177 
178       VersionedProtocol proxy =
179           (VersionedProtocol) Proxy.newProxyInstance(
180               protocol.getClassLoader(), new Class[] { protocol },
181               new Invoker(protocol, addr, ticket, conf, factory, rpcTimeout));
182     if (proxy instanceof VersionedProtocol) {
183       long serverVersion = ((VersionedProtocol)proxy)
184         .getProtocolVersion(protocol.getName(), clientVersion);
185       if (serverVersion != clientVersion) {
186         throw new HBaseRPC.VersionMismatch(protocol.getName(), clientVersion,
187                                       serverVersion);
188       }
189     }
190     return proxy;
191   }
192 
193   /**
194    * Stop this proxy and release its invoker's resource
195    * @param proxy the proxy to be stopped
196    */
197   public void stopProxy(VersionedProtocol proxy) {
198     if (proxy!=null) {
199       ((Invoker)Proxy.getInvocationHandler(proxy)).close();
200     }
201   }
202 
203 
204   /** Expert: Make multiple, parallel calls to a set of servers. */
205   public Object[] call(Method method, Object[][] params,
206                        InetSocketAddress[] addrs,
207                        Class<? extends VersionedProtocol> protocol,
208                        User ticket, Configuration conf)
209     throws IOException, InterruptedException {
210 
211     Invocation[] invocations = new Invocation[params.length];
212     for (int i = 0; i < params.length; i++)
213       invocations[i] = new Invocation(method, params[i]);
214     HBaseClient client = CLIENTS.getClient(conf);
215     try {
216     Writable[] wrappedValues =
217       client.call(invocations, addrs, protocol, ticket);
218 
219     if (method.getReturnType() == Void.TYPE) {
220       return null;
221     }
222 
223     Object[] values =
224       (Object[])Array.newInstance(method.getReturnType(), wrappedValues.length);
225     for (int i = 0; i < values.length; i++)
226       if (wrappedValues[i] != null)
227         values[i] = ((HbaseObjectWritable)wrappedValues[i]).get();
228 
229     return values;
230     } finally {
231       CLIENTS.stopClient(client);
232     }
233   }
234 
235   /** Construct a server for a protocol implementation instance listening on a
236    * port and address. */
237   public Server getServer(Class<? extends VersionedProtocol> protocol,
238                           Object instance,
239                           Class<?>[] ifaces,
240                           String bindAddress, int port,
241                           int numHandlers,
242                           int metaHandlerCount, boolean verbose,
243                           Configuration conf, int highPriorityLevel)
244     throws IOException {
245     return new Server(instance, ifaces, conf, bindAddress, port, numHandlers,
246         metaHandlerCount, verbose, highPriorityLevel);
247   }
248 
249   /** An RPC Server. */
250   public static class Server extends HBaseServer {
251     private Object instance;
252     private Class<?> implementation;
253     private Class<?>[] ifaces;
254     private boolean verbose;
255     private boolean authorize = false;
256 
257     // for JSON encoding
258     private static ObjectMapper mapper = new ObjectMapper();
259 
260     private static final String WARN_RESPONSE_TIME =
261       "hbase.ipc.warn.response.time";
262     private static final String WARN_RESPONSE_SIZE =
263       "hbase.ipc.warn.response.size";
264 
265     /** Default value for above params */
266     private static final int DEFAULT_WARN_RESPONSE_TIME = 10000; // milliseconds
267     private static final int DEFAULT_WARN_RESPONSE_SIZE = 100 * 1024 * 1024;
268 
269     /** Names for suffixed metrics */
270     private static final String ABOVE_ONE_SEC_METRIC = ".aboveOneSec.";
271 
272     private final int warnResponseTime;
273     private final int warnResponseSize;
274 
275     private static String classNameBase(String className) {
276       String[] names = className.split("\\.", -1);
277       if (names == null || names.length == 0) {
278         return className;
279       }
280       return names[names.length-1];
281     }
282 
283     /** Construct an RPC server.
284      * @param instance the instance whose methods will be called
285      * @param conf the configuration to use
286      * @param bindAddress the address to bind on to listen for connection
287      * @param port the port to listen for connections on
288      * @param numHandlers the number of method handler threads to run
289      * @param verbose whether each call should be logged
290      * @throws IOException e
291      */
292     public Server(Object instance, final Class<?>[] ifaces,
293                   Configuration conf, String bindAddress,  int port,
294                   int numHandlers, int metaHandlerCount, boolean verbose,
295                   int highPriorityLevel) throws IOException {
296       super(bindAddress, port, Invocation.class, numHandlers, metaHandlerCount,
297           conf, classNameBase(instance.getClass().getName()),
298           highPriorityLevel);
299       this.instance = instance;
300       this.implementation = instance.getClass();
301       this.verbose = verbose;
302 
303       this.ifaces = ifaces;
304 
305       // create metrics for the advertised interfaces this server implements.
306       String [] metricSuffixes = new String [] {ABOVE_ONE_SEC_METRIC};
307       this.rpcMetrics.createMetrics(this.ifaces, false, metricSuffixes);
308 
309       this.authorize =
310         conf.getBoolean(
311             ServiceAuthorizationManager.SERVICE_AUTHORIZATION_CONFIG, false);
312 
313       this.warnResponseTime = conf.getInt(WARN_RESPONSE_TIME,
314           DEFAULT_WARN_RESPONSE_TIME);
315       this.warnResponseSize = conf.getInt(WARN_RESPONSE_SIZE,
316           DEFAULT_WARN_RESPONSE_SIZE);
317     }
318 
319     @Override
320     public Writable call(Class<? extends VersionedProtocol> protocol,
321         Writable param, long receivedTime, MonitoredRPCHandler status)
322     throws IOException {
323       try {
324         Invocation call = (Invocation)param;
325         if(call.getMethodName() == null) {
326           throw new IOException("Could not find requested method, the usual " +
327               "cause is a version mismatch between client and server.");
328         }
329         if (verbose) log("Call: " + call);
330         status.setRPC(call.getMethodName(), call.getParameters(), receivedTime);
331         status.setRPCPacket(param);
332         status.resume("Servicing call");
333 
334         Method method =
335           protocol.getMethod(call.getMethodName(),
336                                    call.getParameterClasses());
337         method.setAccessible(true);
338 
339         //Verify protocol version.
340         //Bypass the version check for VersionedProtocol
341         if (!method.getDeclaringClass().equals(VersionedProtocol.class)) {
342           long clientVersion = call.getProtocolVersion();
343           ProtocolSignature serverInfo = ((VersionedProtocol) instance)
344               .getProtocolSignature(protocol.getCanonicalName(), call
345                   .getProtocolVersion(), call.getClientMethodsHash());
346           long serverVersion = serverInfo.getVersion();
347           if (serverVersion != clientVersion) {
348             LOG.warn("Version mismatch: client version=" + clientVersion
349                 + ", server version=" + serverVersion);
350             throw new RPC.VersionMismatch(protocol.getName(), clientVersion,
351                 serverVersion);
352           }
353         }
354         Object impl = null;
355         if (protocol.isAssignableFrom(this.implementation)) {
356           impl = this.instance;
357         }
358         else {
359           throw new HBaseRPC.UnknownProtocolException(protocol);
360         }
361 
362         long startTime = System.currentTimeMillis();
363         Object[] params = call.getParameters();
364         Object value = method.invoke(impl, params);
365         int processingTime = (int) (System.currentTimeMillis() - startTime);
366         int qTime = (int) (startTime-receivedTime);
367         if (TRACELOG.isDebugEnabled()) {
368           TRACELOG.debug("Call #" + CurCall.get().id +
369               "; Served: " + protocol.getSimpleName()+"#"+call.getMethodName() +
370               " queueTime=" + qTime +
371               " processingTime=" + processingTime +
372               " contents=" + Objects.describeQuantity(params));
373         }
374         rpcMetrics.rpcQueueTime.inc(qTime);
375         rpcMetrics.rpcProcessingTime.inc(processingTime);
376         rpcMetrics.inc(call.getMethodName(), processingTime);
377         if (verbose) log("Return: "+value);
378 
379         HbaseObjectWritable retVal =
380           new HbaseObjectWritable(method.getReturnType(), value);
381         long responseSize = retVal.getWritableSize();
382         // log any RPC responses that are slower than the configured warn
383         // response time or larger than configured warning size
384         boolean tooSlow = (processingTime > warnResponseTime
385             && warnResponseTime > -1);
386         boolean tooLarge = (responseSize > warnResponseSize
387             && warnResponseSize > -1);
388         if (tooSlow || tooLarge) {
389           // when tagging, we let TooLarge trump TooSmall to keep output simple
390           // note that large responses will often also be slow.
391           logResponse(call, (tooLarge ? "TooLarge" : "TooSlow"),
392               status.getClient(), startTime, processingTime, qTime,
393               responseSize);
394           // provides a count of log-reported slow responses
395           if (tooSlow) {
396             rpcMetrics.rpcSlowResponseTime.inc(processingTime);
397           }
398         }
399         if (processingTime > 1000) {
400           // we use a hard-coded one second period so that we can clearly
401           // indicate the time period we're warning about in the name of the 
402           // metric itself
403           rpcMetrics.inc(call.getMethodName() + ABOVE_ONE_SEC_METRIC,
404               processingTime);
405         }
406 
407         return retVal;
408       } catch (InvocationTargetException e) {
409         Throwable target = e.getTargetException();
410         if (target instanceof IOException) {
411           throw (IOException)target;
412         }
413         IOException ioe = new IOException(target.toString());
414         ioe.setStackTrace(target.getStackTrace());
415         throw ioe;
416       } catch (Throwable e) {
417         if (!(e instanceof IOException)) {
418           LOG.error("Unexpected throwable object ", e);
419         }
420         IOException ioe = new IOException(e.toString());
421         ioe.setStackTrace(e.getStackTrace());
422         throw ioe;
423       }
424     }
425 
426     /**
427      * Logs an RPC response to the LOG file, producing valid JSON objects for
428      * client Operations.
429      * @param call The call to log.
430      * @param tag  The tag that will be used to indicate this event in the log.
431      * @param client          The address of the client who made this call.
432      * @param startTime       The time that the call was initiated, in ms.
433      * @param processingTime  The duration that the call took to run, in ms.
434      * @param qTime           The duration that the call spent on the queue 
435      *                        prior to being initiated, in ms.
436      * @param responseSize    The size in bytes of the response buffer.
437      */
438     private void logResponse(Invocation call, String tag, String clientAddress,
439         long startTime, int processingTime, int qTime, long responseSize)
440       throws IOException {
441       Object params[] = call.getParameters();
442       // for JSON encoding
443       ObjectMapper mapper = new ObjectMapper();
444       // base information that is reported regardless of type of call
445       Map<String, Object> responseInfo = new HashMap<String, Object>();
446       responseInfo.put("starttimems", startTime);
447       responseInfo.put("processingtimems", processingTime);
448       responseInfo.put("queuetimems", qTime);
449       responseInfo.put("responsesize", responseSize);
450       responseInfo.put("client", clientAddress);
451       responseInfo.put("class", instance.getClass().getSimpleName());
452       responseInfo.put("method", call.getMethodName());
453       if (params.length == 2 && instance instanceof HRegionServer &&
454           params[0] instanceof byte[] &&
455           params[1] instanceof Operation) {
456         // if the slow process is a query, we want to log its table as well 
457         // as its own fingerprint
458         byte [] tableName =
459           HRegionInfo.parseRegionName((byte[]) params[0])[0];
460         responseInfo.put("table", Bytes.toStringBinary(tableName));
461         // annotate the response map with operation details
462         responseInfo.putAll(((Operation) params[1]).toMap());
463         // report to the log file
464         LOG.warn("(operation" + tag + "): " +
465             mapper.writeValueAsString(responseInfo));
466       } else if (params.length == 1 && instance instanceof HRegionServer &&
467           params[0] instanceof Operation) {
468         // annotate the response map with operation details
469         responseInfo.putAll(((Operation) params[0]).toMap());
470         // report to the log file
471         LOG.warn("(operation" + tag + "): " +
472             mapper.writeValueAsString(responseInfo));
473       } else {
474         // can't get JSON details, so just report call.toString() along with 
475         // a more generic tag.
476         responseInfo.put("call", call.toString());
477         LOG.warn("(response" + tag + "): " +
478             mapper.writeValueAsString(responseInfo));
479       }
480     }
481   }
482 
483   protected static void log(String value) {
484     String v = value;
485     if (v != null && v.length() > 55)
486       v = v.substring(0, 55)+"...";
487     LOG.info(v);
488   }
489 }