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 com.google.common.base.Function;
24  import org.apache.commons.logging.Log;
25  import org.apache.commons.logging.LogFactory;
26  import org.apache.hadoop.conf.Configurable;
27  import org.apache.hadoop.conf.Configuration;
28  import org.apache.hadoop.hbase.client.RetriesExhaustedException;
29  import org.apache.hadoop.hbase.io.HbaseObjectWritable;
30  import org.apache.hadoop.hbase.util.Bytes;
31  import org.apache.hadoop.io.Writable;
32  import org.apache.hadoop.ipc.VersionedProtocol;
33  import org.apache.hadoop.net.NetUtils;
34  import org.apache.hadoop.security.UserGroupInformation;
35  
36  import javax.net.SocketFactory;
37  import java.io.DataInput;
38  import java.io.DataOutput;
39  import java.io.IOException;
40  import java.lang.reflect.Array;
41  import java.lang.reflect.InvocationHandler;
42  import java.lang.reflect.InvocationTargetException;
43  import java.lang.reflect.Method;
44  import java.lang.reflect.Proxy;
45  import java.net.ConnectException;
46  import java.net.InetSocketAddress;
47  import java.net.SocketTimeoutException;
48  import java.util.HashMap;
49  import java.util.Map;
50  
51  /** A simple RPC mechanism.
52   *
53   * This is a local hbase copy of the hadoop RPC so we can do things like
54   * address HADOOP-414 for hbase-only and try other hbase-specific
55   * optimizations like using our own version of ObjectWritable.  Class has been
56   * renamed to avoid confusing it w/ hadoop versions.
57   * <p>
58   *
59   *
60   * A <i>protocol</i> is a Java interface.  All parameters and return types must
61   * be one of:
62   *
63   * <ul> <li>a primitive type, <code>boolean</code>, <code>byte</code>,
64   * <code>char</code>, <code>short</code>, <code>int</code>, <code>long</code>,
65   * <code>float</code>, <code>double</code>, or <code>void</code>; or</li>
66   *
67   * <li>a {@link String}; or</li>
68   *
69   * <li>a {@link Writable}; or</li>
70   *
71   * <li>an array of the above types</li> </ul>
72   *
73   * All methods in the protocol should throw only IOException.  No field data of
74   * the protocol instance is transmitted.
75   */
76  public class HBaseRPC {
77    // Leave this out in the hadoop ipc package but keep class name.  Do this
78    // so that we dont' get the logging of this class's invocations by doing our
79    // blanket enabling DEBUG on the o.a.h.h. package.
80    protected static final Log LOG =
81      LogFactory.getLog("org.apache.hadoop.ipc.HbaseRPC");
82  
83    private HBaseRPC() {
84      super();
85    }                                  // no public ctor
86  
87  
88    /** A method invocation, including the method name and its parameters.*/
89    public static class Invocation implements Writable, Configurable {
90      private String methodName;
91      @SuppressWarnings("unchecked")
92      private Class[] parameterClasses;
93      private Object[] parameters;
94      private Configuration conf;
95  
96      /** default constructor */
97      public Invocation() {
98        super();
99      }
100 
101     /**
102      * @param method method to call
103      * @param parameters parameters of call
104      */
105     public Invocation(Method method, Object[] parameters) {
106       this.methodName = method.getName();
107       this.parameterClasses = method.getParameterTypes();
108       this.parameters = parameters;
109     }
110 
111     /** @return The name of the method invoked. */
112     public String getMethodName() { return methodName; }
113 
114     /** @return The parameter classes. */
115     @SuppressWarnings("unchecked")
116     public Class[] getParameterClasses() { return parameterClasses; }
117 
118     /** @return The parameter instances. */
119     public Object[] getParameters() { return parameters; }
120 
121     public void readFields(DataInput in) throws IOException {
122       methodName = in.readUTF();
123       parameters = new Object[in.readInt()];
124       parameterClasses = new Class[parameters.length];
125       HbaseObjectWritable objectWritable = new HbaseObjectWritable();
126       for (int i = 0; i < parameters.length; i++) {
127         parameters[i] = HbaseObjectWritable.readObject(in, objectWritable,
128           this.conf);
129         parameterClasses[i] = objectWritable.getDeclaredClass();
130       }
131     }
132 
133     public void write(DataOutput out) throws IOException {
134       out.writeUTF(this.methodName);
135       out.writeInt(parameterClasses.length);
136       for (int i = 0; i < parameterClasses.length; i++) {
137         HbaseObjectWritable.writeObject(out, parameters[i], parameterClasses[i],
138                                    conf);
139       }
140     }
141 
142     @Override
143     public String toString() {
144       StringBuilder buffer = new StringBuilder(256);
145       buffer.append(methodName);
146       buffer.append("(");
147       for (int i = 0; i < parameters.length; i++) {
148         if (i != 0)
149           buffer.append(", ");
150         buffer.append(parameters[i]);
151       }
152       buffer.append(")");
153       return buffer.toString();
154     }
155 
156     public void setConf(Configuration conf) {
157       this.conf = conf;
158     }
159 
160     public Configuration getConf() {
161       return this.conf;
162     }
163   }
164 
165   /* Cache a client using its socket factory as the hash key */
166   static private class ClientCache {
167     private Map<SocketFactory, HBaseClient> clients =
168       new HashMap<SocketFactory, HBaseClient>();
169 
170     protected ClientCache() {}
171 
172     /**
173      * Construct & cache an IPC client with the user-provided SocketFactory
174      * if no cached client exists.
175      *
176      * @param conf Configuration
177      * @param factory socket factory
178      * @return an IPC client
179      */
180     protected synchronized HBaseClient getClient(Configuration conf,
181         SocketFactory factory) {
182       // Construct & cache client.  The configuration is only used for timeout,
183       // and Clients have connection pools.  So we can either (a) lose some
184       // connection pooling and leak sockets, or (b) use the same timeout for all
185       // configurations.  Since the IPC is usually intended globally, not
186       // per-job, we choose (a).
187       HBaseClient client = clients.get(factory);
188       if (client == null) {
189         // Make an hbase client instead of hadoop Client.
190         client = new HBaseClient(HbaseObjectWritable.class, conf, factory);
191         clients.put(factory, client);
192       } else {
193         client.incCount();
194       }
195       return client;
196     }
197 
198     /**
199      * Construct & cache an IPC client with the default SocketFactory
200      * if no cached client exists.
201      *
202      * @param conf Configuration
203      * @return an IPC client
204      */
205     protected synchronized HBaseClient getClient(Configuration conf) {
206       return getClient(conf, SocketFactory.getDefault());
207     }
208 
209     /**
210      * Stop a RPC client connection
211      * A RPC client is closed only when its reference count becomes zero.
212      * @param client client to stop
213      */
214     protected void stopClient(HBaseClient client) {
215       synchronized (this) {
216         client.decCount();
217         if (client.isZeroReference()) {
218           clients.remove(client.getSocketFactory());
219         }
220       }
221       if (client.isZeroReference()) {
222         client.stop();
223       }
224     }
225   }
226 
227   protected final static ClientCache CLIENTS = new ClientCache();
228 
229   private static class Invoker implements InvocationHandler {
230     private InetSocketAddress address;
231     private UserGroupInformation ticket;
232     private HBaseClient client;
233     private boolean isClosed = false;
234 
235     /**
236      * @param address address for invoker
237      * @param ticket ticket
238      * @param conf configuration
239      * @param factory socket factory
240      */
241     public Invoker(InetSocketAddress address, UserGroupInformation ticket,
242                    Configuration conf, SocketFactory factory) {
243       this.address = address;
244       this.ticket = ticket;
245       this.client = CLIENTS.getClient(conf, factory);
246     }
247 
248     public Object invoke(Object proxy, Method method, Object[] args)
249         throws Throwable {
250       final boolean logDebug = LOG.isDebugEnabled();
251       long startTime = 0;
252       if (logDebug) {
253         startTime = System.currentTimeMillis();
254       }
255       HbaseObjectWritable value = (HbaseObjectWritable)
256         client.call(new Invocation(method, args), address, ticket);
257       if (logDebug) {
258         long callTime = System.currentTimeMillis() - startTime;
259         LOG.debug("Call: " + method.getName() + " " + callTime);
260       }
261       return value.get();
262     }
263 
264     /* close the IPC client that's responsible for this invoker's RPCs */
265     synchronized protected void close() {
266       if (!isClosed) {
267         isClosed = true;
268         CLIENTS.stopClient(client);
269       }
270     }
271   }
272 
273   /**
274    * A version mismatch for the RPC protocol.
275    */
276   @SuppressWarnings("serial")
277   public static class VersionMismatch extends IOException {
278     private String interfaceName;
279     private long clientVersion;
280     private long serverVersion;
281 
282     /**
283      * Create a version mismatch exception
284      * @param interfaceName the name of the protocol mismatch
285      * @param clientVersion the client's version of the protocol
286      * @param serverVersion the server's version of the protocol
287      */
288     public VersionMismatch(String interfaceName, long clientVersion,
289                            long serverVersion) {
290       super("Protocol " + interfaceName + " version mismatch. (client = " +
291             clientVersion + ", server = " + serverVersion + ")");
292       this.interfaceName = interfaceName;
293       this.clientVersion = clientVersion;
294       this.serverVersion = serverVersion;
295     }
296 
297     /**
298      * Get the interface name
299      * @return the java class name
300      *          (eg. org.apache.hadoop.mapred.InterTrackerProtocol)
301      */
302     public String getInterfaceName() {
303       return interfaceName;
304     }
305 
306     /**
307      * @return the client's preferred version
308      */
309     public long getClientVersion() {
310       return clientVersion;
311     }
312 
313     /**
314      * @return the server's agreed to version.
315      */
316     public long getServerVersion() {
317       return serverVersion;
318     }
319   }
320 
321   /**
322    * @param protocol protocol interface
323    * @param clientVersion which client version we expect
324    * @param addr address of remote service
325    * @param conf configuration
326    * @param maxAttempts max attempts
327    * @param timeout timeout in milliseconds
328    * @return proxy
329    * @throws IOException e
330    */
331   @SuppressWarnings("unchecked")
332   public static VersionedProtocol waitForProxy(Class protocol,
333                                                long clientVersion,
334                                                InetSocketAddress addr,
335                                                Configuration conf,
336                                                int maxAttempts,
337                                                long timeout
338                                                ) throws IOException {
339     // HBase does limited number of reconnects which is different from hadoop.
340     long startTime = System.currentTimeMillis();
341     IOException ioe;
342     int reconnectAttempts = 0;
343     while (true) {
344       try {
345         return getProxy(protocol, clientVersion, addr, conf);
346       } catch(ConnectException se) {  // namenode has not been started
347         ioe = se;
348         if (maxAttempts >= 0 && ++reconnectAttempts >= maxAttempts) {
349           LOG.info("Server at " + addr + " could not be reached after " +
350             reconnectAttempts + " tries, giving up.");
351           throw new RetriesExhaustedException("Failed setting up proxy to " +
352             addr.toString() + " after attempts=" + reconnectAttempts);
353       }
354       } catch(SocketTimeoutException te) {  // namenode is busy
355         LOG.info("Problem connecting to server: " + addr);
356         ioe = te;
357       }
358       // check if timed out
359       if (System.currentTimeMillis()-timeout >= startTime) {
360         throw ioe;
361       }
362 
363       // wait for retry
364       try {
365         Thread.sleep(1000);
366       } catch (InterruptedException ie) {
367         // IGNORE
368       }
369     }
370   }
371 
372   /**
373    * Construct a client-side proxy object that implements the named protocol,
374    * talking to a server at the named address.
375    *
376    * @param protocol interface
377    * @param clientVersion version we are expecting
378    * @param addr remote address
379    * @param conf configuration
380    * @param factory socket factory
381    * @return proxy
382    * @throws IOException e
383    */
384   public static VersionedProtocol getProxy(Class<?> protocol,
385       long clientVersion, InetSocketAddress addr, Configuration conf,
386       SocketFactory factory) throws IOException {
387     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
388     return getProxy(protocol, clientVersion, addr, ugi, conf, factory);
389   }
390 
391   /**
392    * Construct a client-side proxy object that implements the named protocol,
393    * talking to a server at the named address.
394    *
395    * @param protocol interface
396    * @param clientVersion version we are expecting
397    * @param addr remote address
398    * @param ticket ticket
399    * @param conf configuration
400    * @param factory socket factory
401    * @return proxy
402    * @throws IOException e
403    */
404   public static VersionedProtocol getProxy(Class<?> protocol,
405       long clientVersion, InetSocketAddress addr, UserGroupInformation ticket,
406       Configuration conf, SocketFactory factory)
407   throws IOException {
408     VersionedProtocol proxy =
409         (VersionedProtocol) Proxy.newProxyInstance(
410             protocol.getClassLoader(), new Class[] { protocol },
411             new Invoker(addr, ticket, conf, factory));
412     long serverVersion = proxy.getProtocolVersion(protocol.getName(),
413                                                   clientVersion);
414     if (serverVersion == clientVersion) {
415       return proxy;
416     }
417     throw new VersionMismatch(protocol.getName(), clientVersion,
418                               serverVersion);
419   }
420 
421   /**
422    * Construct a client-side proxy object with the default SocketFactory
423    *
424    * @param protocol interface
425    * @param clientVersion version we are expecting
426    * @param addr remote address
427    * @param conf configuration
428    * @return a proxy instance
429    * @throws IOException e
430    */
431   public static VersionedProtocol getProxy(Class<?> protocol,
432       long clientVersion, InetSocketAddress addr, Configuration conf)
433       throws IOException {
434 
435     return getProxy(protocol, clientVersion, addr, conf, NetUtils
436         .getDefaultSocketFactory(conf));
437   }
438 
439   /**
440    * Stop this proxy and release its invoker's resource
441    * @param proxy the proxy to be stopped
442    */
443   public static void stopProxy(VersionedProtocol proxy) {
444     if (proxy!=null) {
445       ((Invoker)Proxy.getInvocationHandler(proxy)).close();
446     }
447   }
448 
449   /**
450    * Expert: Make multiple, parallel calls to a set of servers.
451    *
452    * @param method method to invoke
453    * @param params array of parameters
454    * @param addrs array of addresses
455    * @param conf configuration
456    * @return values
457    * @throws IOException e
458    */
459   public static Object[] call(Method method, Object[][] params,
460                               InetSocketAddress[] addrs, Configuration conf)
461     throws IOException {
462 
463     Invocation[] invocations = new Invocation[params.length];
464     for (int i = 0; i < params.length; i++)
465       invocations[i] = new Invocation(method, params[i]);
466     HBaseClient client = CLIENTS.getClient(conf);
467     try {
468     Writable[] wrappedValues = client.call(invocations, addrs);
469 
470     if (method.getReturnType() == Void.TYPE) {
471       return null;
472     }
473 
474     Object[] values =
475       (Object[])Array.newInstance(method.getReturnType(), wrappedValues.length);
476     for (int i = 0; i < values.length; i++)
477       if (wrappedValues[i] != null)
478         values[i] = ((HbaseObjectWritable)wrappedValues[i]).get();
479 
480     return values;
481     } finally {
482       CLIENTS.stopClient(client);
483     }
484   }
485 
486   /**
487    * Construct a server for a protocol implementation instance listening on a
488    * port and address.
489    *
490    * @param instance instance
491    * @param bindAddress bind address
492    * @param port port to bind to
493    * @param numHandlers number of handlers to start
494    * @param verbose verbose flag
495    * @param conf configuration
496    * @return Server
497    * @throws IOException e
498    */
499   public static Server getServer(final Object instance,
500                                  final Class<?>[] ifaces,
501                                  final String bindAddress, final int port,
502                                  final int numHandlers,
503                                  int metaHandlerCount, final boolean verbose, Configuration conf, int highPriorityLevel)
504     throws IOException {
505     return new Server(instance, ifaces, conf, bindAddress, port, numHandlers, metaHandlerCount, verbose, highPriorityLevel);
506   }
507 
508   /** An RPC Server. */
509   public static class Server extends HBaseServer {
510     private Object instance;
511     private Class<?> implementation;
512     private Class<?> ifaces[];
513     private boolean verbose;
514 
515     private static String classNameBase(String className) {
516       String[] names = className.split("\\.", -1);
517       if (names == null || names.length == 0) {
518         return className;
519       }
520       return names[names.length-1];
521     }
522 
523     /** Construct an RPC server.
524      * @param instance the instance whose methods will be called
525      * @param conf the configuration to use
526      * @param bindAddress the address to bind on to listen for connection
527      * @param port the port to listen for connections on
528      * @param numHandlers the number of method handler threads to run
529      * @param verbose whether each call should be logged
530      * @throws IOException e
531      */
532     public Server(Object instance, final Class<?>[] ifaces,
533                   Configuration conf, String bindAddress, int port,
534                   int numHandlers, int metaHandlerCount, boolean verbose, int highPriorityLevel) throws IOException {
535       super(bindAddress, port, Invocation.class, numHandlers, metaHandlerCount, conf, classNameBase(instance.getClass().getName()), highPriorityLevel);
536       this.instance = instance;
537       this.implementation = instance.getClass();
538 
539       this.verbose = verbose;
540 
541       this.ifaces = ifaces;
542 
543       // create metrics for the advertised interfaces this server implements.
544       this.rpcMetrics.createMetrics(this.ifaces);
545     }
546 
547     @Override
548     public Writable call(Writable param, long receivedTime) throws IOException {
549       try {
550         Invocation call = (Invocation)param;
551         if(call.getMethodName() == null) {
552           throw new IOException("Could not find requested method, the usual " +
553               "cause is a version mismatch between client and server.");
554         }
555         if (verbose) log("Call: " + call);
556         Method method =
557           implementation.getMethod(call.getMethodName(),
558                                    call.getParameterClasses());
559 
560         long startTime = System.currentTimeMillis();
561         Object value = method.invoke(instance, call.getParameters());
562         int processingTime = (int) (System.currentTimeMillis() - startTime);
563         int qTime = (int) (startTime-receivedTime);
564         if (LOG.isDebugEnabled()) {
565           LOG.debug("Served: " + call.getMethodName() +
566             " queueTime= " + qTime +
567             " procesingTime= " + processingTime);
568         }
569         rpcMetrics.rpcQueueTime.inc(qTime);
570         rpcMetrics.rpcProcessingTime.inc(processingTime);
571         rpcMetrics.inc(call.getMethodName(), processingTime);
572         if (verbose) log("Return: "+value);
573 
574         return new HbaseObjectWritable(method.getReturnType(), value);
575 
576       } catch (InvocationTargetException e) {
577         Throwable target = e.getTargetException();
578         if (target instanceof IOException) {
579           throw (IOException)target;
580         }
581         IOException ioe = new IOException(target.toString());
582         ioe.setStackTrace(target.getStackTrace());
583         throw ioe;
584       } catch (Throwable e) {
585         IOException ioe = new IOException(e.toString());
586         ioe.setStackTrace(e.getStackTrace());
587         throw ioe;
588       }
589     }
590   }
591 
592   protected static void log(String value) {
593     String v = value;
594     if (v != null && v.length() > 55)
595       v = v.substring(0, 55)+"...";
596     LOG.info(v);
597   }
598 }