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  package org.apache.hadoop.hbase.ipc;
19  
20  import static org.apache.hadoop.hbase.ipc.CallEvent.Type.CANCELLED;
21  import static org.apache.hadoop.hbase.ipc.CallEvent.Type.TIMEOUT;
22  import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled;
23  import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE;
24  
25  import com.google.protobuf.RpcCallback;
26  
27  import io.netty.bootstrap.Bootstrap;
28  import io.netty.buffer.ByteBuf;
29  import io.netty.buffer.ByteBufOutputStream;
30  import io.netty.buffer.Unpooled;
31  import io.netty.channel.Channel;
32  import io.netty.channel.ChannelFuture;
33  import io.netty.channel.ChannelFutureListener;
34  import io.netty.channel.ChannelHandler;
35  import io.netty.channel.ChannelOption;
36  import io.netty.channel.ChannelPipeline;
37  import io.netty.handler.codec.LengthFieldBasedFrameDecoder;
38  import io.netty.handler.timeout.IdleStateHandler;
39  import io.netty.util.ReferenceCountUtil;
40  import io.netty.util.concurrent.Future;
41  import io.netty.util.concurrent.FutureListener;
42  import io.netty.util.concurrent.Promise;
43  
44  import java.io.IOException;
45  import java.net.InetSocketAddress;
46  import java.net.UnknownHostException;
47  import java.util.concurrent.Executors;
48  import java.util.concurrent.ScheduledExecutorService;
49  import java.util.concurrent.ThreadLocalRandom;
50  import java.util.concurrent.TimeUnit;
51  
52  import org.apache.commons.logging.Log;
53  import org.apache.commons.logging.LogFactory;
54  import org.apache.hadoop.hbase.classification.InterfaceAudience;
55  import org.apache.hadoop.hbase.ipc.BufferCallBeforeInitHandler.BufferCallEvent;
56  import org.apache.hadoop.hbase.ipc.HBaseRpcController.CancellationCallback;
57  import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
58  import org.apache.hadoop.hbase.security.NettyHBaseSaslRpcClientHandler;
59  import org.apache.hadoop.hbase.security.SaslChallengeDecoder;
60  import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
61  import org.apache.hadoop.hbase.util.Threads;
62  import org.apache.hadoop.security.UserGroupInformation;
63  
64  /**
65   * RPC connection implementation based on netty.
66   * <p>
67   * Most operations are executed in handlers. Netty handler is always executed in the same
68   * thread(EventLoop) so no lock is needed.
69   */
70  @InterfaceAudience.Private
71  class NettyRpcConnection extends RpcConnection {
72  
73    private static final Log LOG = LogFactory.getLog(NettyRpcConnection.class);
74  
75    private static final ScheduledExecutorService RELOGIN_EXECUTOR =
76        Executors.newSingleThreadScheduledExecutor(Threads.newDaemonThreadFactory("Relogin"));
77  
78    private final NettyRpcClient rpcClient;
79  
80    private ByteBuf connectionHeaderPreamble;
81  
82    private ByteBuf connectionHeaderWithLength;
83  
84    @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC",
85        justification = "connect is also under lock as notifyOnCancel will call our action directly")
86    private Channel channel;
87  
88    NettyRpcConnection(NettyRpcClient rpcClient, ConnectionId remoteId) throws IOException {
89      super(rpcClient.conf, AbstractRpcClient.WHEEL_TIMER, remoteId, rpcClient.clusterId,
90          rpcClient.userProvider.isHBaseSecurityEnabled(), rpcClient.codec, rpcClient.compressor,
91          rpcClient.metrics);
92      this.rpcClient = rpcClient;
93      byte[] connectionHeaderPreamble = getConnectionHeaderPreamble();
94      this.connectionHeaderPreamble =
95          Unpooled.directBuffer(connectionHeaderPreamble.length).writeBytes(connectionHeaderPreamble);
96      ConnectionHeader header = getConnectionHeader();
97      this.connectionHeaderWithLength = Unpooled.directBuffer(4 + header.getSerializedSize());
98      this.connectionHeaderWithLength.writeInt(header.getSerializedSize());
99      header.writeTo(new ByteBufOutputStream(this.connectionHeaderWithLength));
100   }
101 
102   @Override
103   protected synchronized void callTimeout(Call call) {
104     if (channel != null) {
105       channel.pipeline().fireUserEventTriggered(new CallEvent(TIMEOUT, call));
106     }
107   }
108 
109   @Override
110   public synchronized boolean isActive() {
111     return channel != null;
112   }
113 
114   private void shutdown0() {
115     if (channel != null) {
116       channel.close();
117       channel = null;
118     }
119   }
120 
121   @Override
122   public synchronized void shutdown() {
123     shutdown0();
124   }
125 
126   @Override
127   public synchronized void cleanupConnection() {
128     if (connectionHeaderPreamble != null) {
129       ReferenceCountUtil.safeRelease(connectionHeaderPreamble);
130     }
131     if (connectionHeaderWithLength != null) {
132       ReferenceCountUtil.safeRelease(connectionHeaderWithLength);
133     }
134   }
135 
136   private void established(Channel ch) {
137     ch.write(connectionHeaderWithLength.retainedDuplicate());
138     ChannelPipeline p = ch.pipeline();
139     String addBeforeHandler = p.context(BufferCallBeforeInitHandler.class).name();
140     p.addBefore(addBeforeHandler, null,
141       new IdleStateHandler(0, rpcClient.minIdleTimeBeforeClose, 0, TimeUnit.MILLISECONDS));
142     p.addBefore(addBeforeHandler, null, new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4));
143     p.addBefore(addBeforeHandler, null,
144       new NettyRpcDuplexHandler(this, rpcClient.cellBlockBuilder, codec, compressor));
145     p.fireUserEventTriggered(BufferCallEvent.success());
146   }
147 
148   private boolean reloginInProgress;
149 
150   private void scheduleRelogin(Throwable error) {
151     if (error instanceof FallbackDisallowedException) {
152       return;
153     }
154     synchronized (this) {
155       if (reloginInProgress) {
156         return;
157       }
158       reloginInProgress = true;
159       RELOGIN_EXECUTOR.schedule(new Runnable() {
160 
161         @Override
162         public void run() {
163           try {
164             if (shouldAuthenticateOverKrb()) {
165               relogin();
166             }
167           } catch (IOException e) {
168             LOG.warn("relogin failed", e);
169           }
170           synchronized (this) {
171             reloginInProgress = false;
172           }
173         }
174       }, ThreadLocalRandom.current().nextInt(reloginMaxBackoff), TimeUnit.MILLISECONDS);
175     }
176   }
177 
178   private void failInit(Channel ch, IOException e) {
179     synchronized (this) {
180       // fail all pending calls
181       ch.pipeline().fireUserEventTriggered(BufferCallEvent.fail(e));
182       shutdown0();
183       return;
184     }
185   }
186 
187   private void saslNegotiate(final Channel ch) {
188     UserGroupInformation ticket = getUGI();
189     if (ticket == null) {
190       failInit(ch, new FatalConnectionException("ticket/user is null"));
191       return;
192     }
193     Promise<Boolean> saslPromise = ch.eventLoop().newPromise();
194     ChannelHandler saslHandler;
195     try {
196       saslHandler = new NettyHBaseSaslRpcClientHandler(saslPromise, ticket, authMethod, token,
197           serverPrincipal, rpcClient.fallbackAllowed, this.rpcClient.conf.get(
198             "hbase.rpc.protection", QualityOfProtection.AUTHENTICATION.name().toLowerCase()));
199     } catch (IOException e) {
200       failInit(ch, e);
201       return;
202     }
203     ch.pipeline().addFirst(new SaslChallengeDecoder(), saslHandler);
204     saslPromise.addListener(new FutureListener<Boolean>() {
205 
206       @Override
207       public void operationComplete(Future<Boolean> future) throws Exception {
208         if (future.isSuccess()) {
209           ChannelPipeline p = ch.pipeline();
210           p.remove(SaslChallengeDecoder.class);
211           p.remove(NettyHBaseSaslRpcClientHandler.class);
212           established(ch);
213         } else {
214           final Throwable error = future.cause();
215           scheduleRelogin(error);
216           failInit(ch, toIOE(error));
217         }
218       }
219     });
220   }
221 
222   private void connect() throws UnknownHostException {
223     if (LOG.isDebugEnabled()) {
224       LOG.debug("Connecting to " + remoteId.getAddress());
225     }
226     if (this.rpcClient.metrics != null) {
227       this.rpcClient.metrics.incrNsLookups();
228     }
229     InetSocketAddress remoteAddr = remoteId.getAddress().toSocketAddress();
230     if (remoteAddr.isUnresolved()) {
231       if (this.rpcClient.metrics != null) {
232         this.rpcClient.metrics.incrNsLookupsFailed();
233       }
234       throw new UnknownHostException(remoteId.getAddress() + " could not be resolved");
235     }
236     this.channel = new Bootstrap().group(rpcClient.group).channel(rpcClient.channelClass)
237         .option(ChannelOption.TCP_NODELAY, rpcClient.isTcpNoDelay())
238         .option(ChannelOption.SO_KEEPALIVE, rpcClient.tcpKeepAlive)
239         .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, rpcClient.connectTO)
240         .handler(new BufferCallBeforeInitHandler()).localAddress(rpcClient.localAddr)
241         .remoteAddress(remoteAddr).connect().addListener(new ChannelFutureListener() {
242 
243           @Override
244           public void operationComplete(ChannelFuture future) throws Exception {
245             Channel ch = future.channel();
246             if (!future.isSuccess()) {
247               failInit(ch, toIOE(future.cause()));
248               rpcClient.failedServers.addToFailedServers(remoteId.getAddress(), future.cause());
249               return;
250             }
251             ch.writeAndFlush(connectionHeaderPreamble.retainedDuplicate());
252             if (useSasl) {
253               saslNegotiate(ch);
254             } else {
255               established(ch);
256             }
257           }
258         }).channel();
259   }
260 
261   private void write(Channel ch, final Call call) {
262     ch.writeAndFlush(call).addListener(new ChannelFutureListener() {
263 
264       @Override
265       public void operationComplete(ChannelFuture future) throws Exception {
266         // Fail the call if we failed to write it out. This usually because the channel is
267         // closed. This is needed because we may shutdown the channel inside event loop and
268         // there may still be some pending calls in the event loop queue after us.
269         if (!future.isSuccess()) {
270           call.setException(toIOE(future.cause()));
271         }
272       }
273     });
274   }
275 
276   @Override
277   public synchronized void sendRequest(final Call call, HBaseRpcController hrc) throws IOException {
278     if (reloginInProgress) {
279       throw new IOException("Can not send request because relogin is in progress.");
280     }
281     hrc.notifyOnCancel(new RpcCallback<Object>() {
282 
283       @Override
284       public void run(Object parameter) {
285         setCancelled(call);
286         synchronized (this) {
287           if (channel != null) {
288             channel.pipeline().fireUserEventTriggered(new CallEvent(CANCELLED, call));
289           }
290         }
291       }
292     }, new CancellationCallback() {
293 
294       @Override
295       public void run(boolean cancelled) throws IOException {
296         if (cancelled) {
297           setCancelled(call);
298         } else {
299           if (channel == null) {
300             connect();
301           }
302           scheduleTimeoutTask(call);
303           final Channel ch = channel;
304           // We must move the whole writeAndFlush call inside event loop otherwise there will be a
305           // race condition.
306           // In netty's DefaultChannelPipeline, it will find the first outbound handler in the
307           // current thread and then schedule a task to event loop which will start the process from
308           // that outbound handler. It is possible that the first handler is
309           // BufferCallBeforeInitHandler when we call writeAndFlush here, but the connection is set
310           // up at the same time so in the event loop thread we remove the
311           // BufferCallBeforeInitHandler, and then our writeAndFlush task comes, still calls the
312           // write method of BufferCallBeforeInitHandler.
313           // This may be considered as a bug of netty, but anyway there is a work around so let's
314           // fix it by ourselves first.
315           if (ch.eventLoop().inEventLoop()) {
316             write(ch, call);
317           } else {
318             ch.eventLoop().execute(new Runnable() {
319 
320               @Override
321               public void run() {
322                 write(ch, call);
323               }
324             });
325           }
326         }
327       }
328     });
329   }
330 }