View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  package org.apache.hadoop.hbase.ipc;
21  
22  import com.google.protobuf.Message;
23  import com.google.protobuf.ServiceException;
24  import org.apache.commons.logging.Log;
25  import org.apache.commons.logging.LogFactory;
26  import org.apache.hadoop.conf.Configuration;
27  import org.apache.hadoop.hbase.CellScanner;
28  import org.apache.hadoop.hbase.IpcProtocol;
29  import org.apache.hadoop.hbase.security.User;
30  import org.apache.hadoop.hbase.util.Pair;
31  import org.apache.hadoop.ipc.RemoteException;
32  
33  import java.io.IOException;
34  import java.lang.reflect.InvocationHandler;
35  import java.lang.reflect.Method;
36  import java.lang.reflect.Proxy;
37  import java.net.InetSocketAddress;
38  import java.util.Map;
39  import java.util.concurrent.ConcurrentHashMap;
40  
41  public class ProtobufRpcClientEngine implements RpcClientEngine {
42  
43    private static final Log LOG =
44        LogFactory.getLog("org.apache.hadoop.hbase.ipc.ProtobufRpcClientEngine");
45  
46    public HBaseClient getClient() {
47      return client;
48    }
49  
50    protected HBaseClient client;
51  
52    public ProtobufRpcClientEngine(Configuration conf, String clusterId) {
53      this.client = new HBaseClient(conf, clusterId);
54    }
55  
56  
57    @Override
58    public <T extends IpcProtocol> T getProxy(
59        Class<T> protocol, InetSocketAddress addr,
60        Configuration conf, int rpcTimeout) throws IOException {
61      final Invoker invoker = new Invoker(protocol, addr, User.getCurrent(), rpcTimeout, client);
62      return (T) Proxy.newProxyInstance(
63          protocol.getClassLoader(), new Class[]{protocol}, invoker);
64    }
65  
66    @Override
67    public void close() {
68      this.client.stop();
69    }
70  
71    static class Invoker implements InvocationHandler {
72      private static final Map<String, Message> returnTypes =
73          new ConcurrentHashMap<String, Message>();
74      private Class<? extends IpcProtocol> protocol;
75      private InetSocketAddress address;
76      private User ticket;
77      private HBaseClient client;
78      final private int rpcTimeout;
79  
80      public Invoker(Class<? extends IpcProtocol> protocol, InetSocketAddress addr, User ticket,
81          int rpcTimeout, HBaseClient client)
82      throws IOException {
83        this.protocol = protocol;
84        this.address = addr;
85        this.ticket = ticket;
86        this.client = client;
87        this.rpcTimeout = rpcTimeout;
88      }
89  
90      /**
91       * This is the client side invoker of RPC method. It only throws
92       * ServiceException, since the invocation proxy expects only
93       * ServiceException to be thrown by the method in case protobuf service.
94       *
95       * ServiceException has the following causes:
96       * <ol>
97       * <li>Exceptions encountered on the client side in this method are
98       * set as cause in ServiceException as is.</li>
99       * <li>Exceptions from the server are wrapped in RemoteException and are
100      * set as cause in ServiceException</li>
101      * </ol>
102      *
103      * <p>Note that the client calling protobuf RPC methods, must handle
104      * ServiceException by getting the cause from the ServiceException. If the
105      * cause is RemoteException, then unwrap it to get the exception thrown by
106      * the server.
107      */
108     @Override
109     public Object invoke(Object proxy, Method method, Object[] args)
110     throws ServiceException {
111       long startTime = 0;
112       if (LOG.isTraceEnabled()) {
113         startTime = System.currentTimeMillis();
114       }
115       if (args.length != 2) {
116         throw new ServiceException(method.getName() + " didn't get two args: " + args.length);
117       }
118       // Get the controller.  Often null.  Presume payload carrying controller.  Payload is optional.
119       // It is cells/data that we do not want to protobuf.
120       PayloadCarryingRpcController controller = (PayloadCarryingRpcController)args[0];
121       CellScanner cells = null;
122       if (controller != null) {
123         cells = controller.cellScanner();
124         // Clear it here so we don't by mistake try and these cells processing results.
125         controller.setCellScanner(null);
126       }
127       // The request parameter
128       Message param = (Message)args[1];
129       Pair<Message, CellScanner> val = null;
130       try {
131         val = client.call(method, param, cells, address, protocol, ticket, rpcTimeout);
132         if (controller != null) {
133           // Shove the results into controller so can be carried across the proxy/pb service void.
134           if (val.getSecond() != null) controller.setCellScanner(val.getSecond());
135         } else if (val.getSecond() != null) {
136           throw new ServiceException("Client dropping data on the floor!");
137         }
138 
139         if (LOG.isTraceEnabled()) {
140           long callTime = System.currentTimeMillis() - startTime;
141           if (LOG.isTraceEnabled()) LOG.trace("Call: " + method.getName() + " " + callTime);
142         }
143         return val.getFirst();
144       } catch (Throwable e) {
145         if (e instanceof RemoteException) {
146           Throwable cause = ((RemoteException)e).unwrapRemoteException();
147           throw new ServiceException("methodName=" + method.getName(), cause);
148         }
149         throw new ServiceException(e);
150       }
151     }
152 
153     static Message getReturnProtoType(Method method) throws Exception {
154       if (returnTypes.containsKey(method.getName())) {
155         return returnTypes.get(method.getName());
156       }
157       Class<?> returnType = method.getReturnType();
158       if (returnType.getName().equals("void")) return null;
159       Method newInstMethod = returnType.getMethod("getDefaultInstance");
160       newInstMethod.setAccessible(true);
161       Message protoType = (Message) newInstMethod.invoke(null, (Object[]) null);
162       returnTypes.put(method.getName(), protoType);
163       return protoType;
164     }
165   }
166 }