| <!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd"> |
| <html lang="en"> |
| <head> |
| <title>Source code</title> |
| <link rel="stylesheet" type="text/css" href="../../../../../../stylesheet.css" title="Style"> |
| </head> |
| <body> |
| <div class="sourceContainer"> |
| <pre><span class="sourceLineNo">001</span>/*<a name="line.1"></a> |
| <span class="sourceLineNo">002</span> * Licensed to the Apache Software Foundation (ASF) under one<a name="line.2"></a> |
| <span class="sourceLineNo">003</span> * or more contributor license agreements. See the NOTICE file<a name="line.3"></a> |
| <span class="sourceLineNo">004</span> * distributed with this work for additional information<a name="line.4"></a> |
| <span class="sourceLineNo">005</span> * regarding copyright ownership. The ASF licenses this file<a name="line.5"></a> |
| <span class="sourceLineNo">006</span> * to you under the Apache License, Version 2.0 (the<a name="line.6"></a> |
| <span class="sourceLineNo">007</span> * "License"); you may not use this file except in compliance<a name="line.7"></a> |
| <span class="sourceLineNo">008</span> * with the License. You may obtain a copy of the License at<a name="line.8"></a> |
| <span class="sourceLineNo">009</span> *<a name="line.9"></a> |
| <span class="sourceLineNo">010</span> * http://www.apache.org/licenses/LICENSE-2.0<a name="line.10"></a> |
| <span class="sourceLineNo">011</span> *<a name="line.11"></a> |
| <span class="sourceLineNo">012</span> * Unless required by applicable law or agreed to in writing, software<a name="line.12"></a> |
| <span class="sourceLineNo">013</span> * distributed under the License is distributed on an "AS IS" BASIS,<a name="line.13"></a> |
| <span class="sourceLineNo">014</span> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.<a name="line.14"></a> |
| <span class="sourceLineNo">015</span> * See the License for the specific language governing permissions and<a name="line.15"></a> |
| <span class="sourceLineNo">016</span> * limitations under the License.<a name="line.16"></a> |
| <span class="sourceLineNo">017</span> */<a name="line.17"></a> |
| <span class="sourceLineNo">018</span>package org.apache.hadoop.hbase.ipc;<a name="line.18"></a> |
| <span class="sourceLineNo">019</span><a name="line.19"></a> |
| <span class="sourceLineNo">020</span>import static org.apache.hadoop.hbase.ipc.IPCUtil.buildRequestHeader;<a name="line.20"></a> |
| <span class="sourceLineNo">021</span>import static org.apache.hadoop.hbase.ipc.IPCUtil.createRemoteException;<a name="line.21"></a> |
| <span class="sourceLineNo">022</span>import static org.apache.hadoop.hbase.ipc.IPCUtil.getTotalSizeWhenWrittenDelimited;<a name="line.22"></a> |
| <span class="sourceLineNo">023</span>import static org.apache.hadoop.hbase.ipc.IPCUtil.isFatalConnectionException;<a name="line.23"></a> |
| <span class="sourceLineNo">024</span>import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled;<a name="line.24"></a> |
| <span class="sourceLineNo">025</span>import static org.apache.hadoop.hbase.ipc.IPCUtil.write;<a name="line.25"></a> |
| <span class="sourceLineNo">026</span><a name="line.26"></a> |
| <span class="sourceLineNo">027</span>import io.opentelemetry.context.Scope;<a name="line.27"></a> |
| <span class="sourceLineNo">028</span>import java.io.BufferedInputStream;<a name="line.28"></a> |
| <span class="sourceLineNo">029</span>import java.io.BufferedOutputStream;<a name="line.29"></a> |
| <span class="sourceLineNo">030</span>import java.io.DataInputStream;<a name="line.30"></a> |
| <span class="sourceLineNo">031</span>import java.io.DataOutputStream;<a name="line.31"></a> |
| <span class="sourceLineNo">032</span>import java.io.IOException;<a name="line.32"></a> |
| <span class="sourceLineNo">033</span>import java.io.InputStream;<a name="line.33"></a> |
| <span class="sourceLineNo">034</span>import java.io.InterruptedIOException;<a name="line.34"></a> |
| <span class="sourceLineNo">035</span>import java.io.OutputStream;<a name="line.35"></a> |
| <span class="sourceLineNo">036</span>import java.net.InetSocketAddress;<a name="line.36"></a> |
| <span class="sourceLineNo">037</span>import java.net.Socket;<a name="line.37"></a> |
| <span class="sourceLineNo">038</span>import java.net.SocketTimeoutException;<a name="line.38"></a> |
| <span class="sourceLineNo">039</span>import java.security.PrivilegedExceptionAction;<a name="line.39"></a> |
| <span class="sourceLineNo">040</span>import java.util.ArrayDeque;<a name="line.40"></a> |
| <span class="sourceLineNo">041</span>import java.util.Locale;<a name="line.41"></a> |
| <span class="sourceLineNo">042</span>import java.util.Queue;<a name="line.42"></a> |
| <span class="sourceLineNo">043</span>import java.util.concurrent.ConcurrentHashMap;<a name="line.43"></a> |
| <span class="sourceLineNo">044</span>import java.util.concurrent.ConcurrentMap;<a name="line.44"></a> |
| <span class="sourceLineNo">045</span>import java.util.concurrent.ThreadLocalRandom;<a name="line.45"></a> |
| <span class="sourceLineNo">046</span>import javax.security.sasl.SaslException;<a name="line.46"></a> |
| <span class="sourceLineNo">047</span>import org.apache.hadoop.conf.Configuration;<a name="line.47"></a> |
| <span class="sourceLineNo">048</span>import org.apache.hadoop.hbase.CellScanner;<a name="line.48"></a> |
| <span class="sourceLineNo">049</span>import org.apache.hadoop.hbase.DoNotRetryIOException;<a name="line.49"></a> |
| <span class="sourceLineNo">050</span>import org.apache.hadoop.hbase.exceptions.ConnectionClosingException;<a name="line.50"></a> |
| <span class="sourceLineNo">051</span>import org.apache.hadoop.hbase.io.ByteArrayOutputStream;<a name="line.51"></a> |
| <span class="sourceLineNo">052</span>import org.apache.hadoop.hbase.ipc.HBaseRpcController.CancellationCallback;<a name="line.52"></a> |
| <span class="sourceLineNo">053</span>import org.apache.hadoop.hbase.log.HBaseMarkers;<a name="line.53"></a> |
| <span class="sourceLineNo">054</span>import org.apache.hadoop.hbase.security.HBaseSaslRpcClient;<a name="line.54"></a> |
| <span class="sourceLineNo">055</span>import org.apache.hadoop.hbase.security.SaslUtil;<a name="line.55"></a> |
| <span class="sourceLineNo">056</span>import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;<a name="line.56"></a> |
| <span class="sourceLineNo">057</span>import org.apache.hadoop.hbase.security.provider.SaslClientAuthenticationProvider;<a name="line.57"></a> |
| <span class="sourceLineNo">058</span>import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;<a name="line.58"></a> |
| <span class="sourceLineNo">059</span>import org.apache.hadoop.hbase.util.ExceptionUtil;<a name="line.59"></a> |
| <span class="sourceLineNo">060</span>import org.apache.hadoop.io.IOUtils;<a name="line.60"></a> |
| <span class="sourceLineNo">061</span>import org.apache.hadoop.ipc.RemoteException;<a name="line.61"></a> |
| <span class="sourceLineNo">062</span>import org.apache.hadoop.net.NetUtils;<a name="line.62"></a> |
| <span class="sourceLineNo">063</span>import org.apache.hadoop.security.UserGroupInformation;<a name="line.63"></a> |
| <span class="sourceLineNo">064</span>import org.apache.hadoop.util.StringUtils;<a name="line.64"></a> |
| <span class="sourceLineNo">065</span>import org.apache.yetus.audience.InterfaceAudience;<a name="line.65"></a> |
| <span class="sourceLineNo">066</span>import org.slf4j.Logger;<a name="line.66"></a> |
| <span class="sourceLineNo">067</span>import org.slf4j.LoggerFactory;<a name="line.67"></a> |
| <span class="sourceLineNo">068</span><a name="line.68"></a> |
| <span class="sourceLineNo">069</span>import org.apache.hbase.thirdparty.com.google.protobuf.Message;<a name="line.69"></a> |
| <span class="sourceLineNo">070</span>import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;<a name="line.70"></a> |
| <span class="sourceLineNo">071</span>import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf;<a name="line.71"></a> |
| <span class="sourceLineNo">072</span>import org.apache.hbase.thirdparty.io.netty.buffer.PooledByteBufAllocator;<a name="line.72"></a> |
| <span class="sourceLineNo">073</span><a name="line.73"></a> |
| <span class="sourceLineNo">074</span>import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;<a name="line.74"></a> |
| <span class="sourceLineNo">075</span>import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos;<a name="line.75"></a> |
| <span class="sourceLineNo">076</span>import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta;<a name="line.76"></a> |
| <span class="sourceLineNo">077</span>import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader;<a name="line.77"></a> |
| <span class="sourceLineNo">078</span>import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse;<a name="line.78"></a> |
| <span class="sourceLineNo">079</span>import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader;<a name="line.79"></a> |
| <span class="sourceLineNo">080</span>import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader;<a name="line.80"></a> |
| <span class="sourceLineNo">081</span><a name="line.81"></a> |
| <span class="sourceLineNo">082</span>/**<a name="line.82"></a> |
| <span class="sourceLineNo">083</span> * Thread that reads responses and notifies callers. Each connection owns a socket connected to a<a name="line.83"></a> |
| <span class="sourceLineNo">084</span> * remote address. Calls are multiplexed through this socket: responses may be delivered out of<a name="line.84"></a> |
| <span class="sourceLineNo">085</span> * order.<a name="line.85"></a> |
| <span class="sourceLineNo">086</span> */<a name="line.86"></a> |
| <span class="sourceLineNo">087</span>@InterfaceAudience.Private<a name="line.87"></a> |
| <span class="sourceLineNo">088</span>class BlockingRpcConnection extends RpcConnection implements Runnable {<a name="line.88"></a> |
| <span class="sourceLineNo">089</span><a name="line.89"></a> |
| <span class="sourceLineNo">090</span> private static final Logger LOG = LoggerFactory.getLogger(BlockingRpcConnection.class);<a name="line.90"></a> |
| <span class="sourceLineNo">091</span><a name="line.91"></a> |
| <span class="sourceLineNo">092</span> private final BlockingRpcClient rpcClient;<a name="line.92"></a> |
| <span class="sourceLineNo">093</span><a name="line.93"></a> |
| <span class="sourceLineNo">094</span> private final String threadName;<a name="line.94"></a> |
| <span class="sourceLineNo">095</span> @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC",<a name="line.95"></a> |
| <span class="sourceLineNo">096</span> justification = "We are always under lock actually")<a name="line.96"></a> |
| <span class="sourceLineNo">097</span> private Thread thread;<a name="line.97"></a> |
| <span class="sourceLineNo">098</span><a name="line.98"></a> |
| <span class="sourceLineNo">099</span> // connected socket. protected for writing UT.<a name="line.99"></a> |
| <span class="sourceLineNo">100</span> protected Socket socket = null;<a name="line.100"></a> |
| <span class="sourceLineNo">101</span> private DataInputStream in;<a name="line.101"></a> |
| <span class="sourceLineNo">102</span> private DataOutputStream out;<a name="line.102"></a> |
| <span class="sourceLineNo">103</span><a name="line.103"></a> |
| <span class="sourceLineNo">104</span> private HBaseSaslRpcClient saslRpcClient;<a name="line.104"></a> |
| <span class="sourceLineNo">105</span><a name="line.105"></a> |
| <span class="sourceLineNo">106</span> // currently active calls<a name="line.106"></a> |
| <span class="sourceLineNo">107</span> private final ConcurrentMap<Integer, Call> calls = new ConcurrentHashMap<>();<a name="line.107"></a> |
| <span class="sourceLineNo">108</span><a name="line.108"></a> |
| <span class="sourceLineNo">109</span> private final CallSender callSender;<a name="line.109"></a> |
| <span class="sourceLineNo">110</span><a name="line.110"></a> |
| <span class="sourceLineNo">111</span> private boolean closed = false;<a name="line.111"></a> |
| <span class="sourceLineNo">112</span><a name="line.112"></a> |
| <span class="sourceLineNo">113</span> private byte[] connectionHeaderPreamble;<a name="line.113"></a> |
| <span class="sourceLineNo">114</span><a name="line.114"></a> |
| <span class="sourceLineNo">115</span> private byte[] connectionHeaderWithLength;<a name="line.115"></a> |
| <span class="sourceLineNo">116</span><a name="line.116"></a> |
| <span class="sourceLineNo">117</span> private boolean waitingConnectionHeaderResponse = false;<a name="line.117"></a> |
| <span class="sourceLineNo">118</span><a name="line.118"></a> |
| <span class="sourceLineNo">119</span> /**<a name="line.119"></a> |
| <span class="sourceLineNo">120</span> * If the client wants to interrupt its calls easily (i.e. call Thread#interrupt), it gets into a<a name="line.120"></a> |
| <span class="sourceLineNo">121</span> * java issue: an interruption during a write closes the socket/channel. A way to avoid this is to<a name="line.121"></a> |
| <span class="sourceLineNo">122</span> * use a different thread for writing. This way, on interruptions, we either cancel the writes or<a name="line.122"></a> |
| <span class="sourceLineNo">123</span> * ignore the answer if the write is already done, but we don't stop the write in the middle. This<a name="line.123"></a> |
| <span class="sourceLineNo">124</span> * adds a thread per region server in the client, so it's kept as an option.<a name="line.124"></a> |
| <span class="sourceLineNo">125</span> * <p><a name="line.125"></a> |
| <span class="sourceLineNo">126</span> * The implementation is simple: the client threads adds their call to the queue, and then wait<a name="line.126"></a> |
| <span class="sourceLineNo">127</span> * for an answer. The CallSender blocks on the queue, and writes the calls one after the other. On<a name="line.127"></a> |
| <span class="sourceLineNo">128</span> * interruption, the client cancels its call. The CallSender checks that the call has not been<a name="line.128"></a> |
| <span class="sourceLineNo">129</span> * canceled before writing it.<a name="line.129"></a> |
| <span class="sourceLineNo">130</span> * </p><a name="line.130"></a> |
| <span class="sourceLineNo">131</span> * When the connection closes, all the calls not yet sent are dismissed. The client thread is<a name="line.131"></a> |
| <span class="sourceLineNo">132</span> * notified with an appropriate exception, as if the call was already sent but the answer not yet<a name="line.132"></a> |
| <span class="sourceLineNo">133</span> * received.<a name="line.133"></a> |
| <span class="sourceLineNo">134</span> * </p><a name="line.134"></a> |
| <span class="sourceLineNo">135</span> */<a name="line.135"></a> |
| <span class="sourceLineNo">136</span> private class CallSender extends Thread {<a name="line.136"></a> |
| <span class="sourceLineNo">137</span><a name="line.137"></a> |
| <span class="sourceLineNo">138</span> private final Queue<Call> callsToWrite;<a name="line.138"></a> |
| <span class="sourceLineNo">139</span><a name="line.139"></a> |
| <span class="sourceLineNo">140</span> private final int maxQueueSize;<a name="line.140"></a> |
| <span class="sourceLineNo">141</span><a name="line.141"></a> |
| <span class="sourceLineNo">142</span> public CallSender(String name, Configuration conf) {<a name="line.142"></a> |
| <span class="sourceLineNo">143</span> int queueSize = conf.getInt("hbase.ipc.client.write.queueSize", 1000);<a name="line.143"></a> |
| <span class="sourceLineNo">144</span> callsToWrite = new ArrayDeque<>(queueSize);<a name="line.144"></a> |
| <span class="sourceLineNo">145</span> this.maxQueueSize = queueSize;<a name="line.145"></a> |
| <span class="sourceLineNo">146</span> setDaemon(true);<a name="line.146"></a> |
| <span class="sourceLineNo">147</span> setName(name + " - writer");<a name="line.147"></a> |
| <span class="sourceLineNo">148</span> }<a name="line.148"></a> |
| <span class="sourceLineNo">149</span><a name="line.149"></a> |
| <span class="sourceLineNo">150</span> public void sendCall(final Call call) throws IOException {<a name="line.150"></a> |
| <span class="sourceLineNo">151</span> if (callsToWrite.size() >= maxQueueSize) {<a name="line.151"></a> |
| <span class="sourceLineNo">152</span> throw new IOException("Can't add " + call.toShortString()<a name="line.152"></a> |
| <span class="sourceLineNo">153</span> + " to the write queue. callsToWrite.size()=" + callsToWrite.size());<a name="line.153"></a> |
| <span class="sourceLineNo">154</span> }<a name="line.154"></a> |
| <span class="sourceLineNo">155</span> callsToWrite.offer(call);<a name="line.155"></a> |
| <span class="sourceLineNo">156</span> BlockingRpcConnection.this.notifyAll();<a name="line.156"></a> |
| <span class="sourceLineNo">157</span> }<a name="line.157"></a> |
| <span class="sourceLineNo">158</span><a name="line.158"></a> |
| <span class="sourceLineNo">159</span> public void remove(Call call) {<a name="line.159"></a> |
| <span class="sourceLineNo">160</span> callsToWrite.remove(call);<a name="line.160"></a> |
| <span class="sourceLineNo">161</span> // By removing the call from the expected call list, we make the list smaller, but<a name="line.161"></a> |
| <span class="sourceLineNo">162</span> // it means as well that we don't know how many calls we cancelled.<a name="line.162"></a> |
| <span class="sourceLineNo">163</span> calls.remove(call.id);<a name="line.163"></a> |
| <span class="sourceLineNo">164</span> call.setException(new CallCancelledException(call.toShortString() + ", waitTime="<a name="line.164"></a> |
| <span class="sourceLineNo">165</span> + (EnvironmentEdgeManager.currentTime() - call.getStartTime()) + ", rpcTimeout="<a name="line.165"></a> |
| <span class="sourceLineNo">166</span> + call.timeout));<a name="line.166"></a> |
| <span class="sourceLineNo">167</span> }<a name="line.167"></a> |
| <span class="sourceLineNo">168</span><a name="line.168"></a> |
| <span class="sourceLineNo">169</span> /**<a name="line.169"></a> |
| <span class="sourceLineNo">170</span> * Reads the call from the queue, write them on the socket.<a name="line.170"></a> |
| <span class="sourceLineNo">171</span> */<a name="line.171"></a> |
| <span class="sourceLineNo">172</span> @Override<a name="line.172"></a> |
| <span class="sourceLineNo">173</span> public void run() {<a name="line.173"></a> |
| <span class="sourceLineNo">174</span> synchronized (BlockingRpcConnection.this) {<a name="line.174"></a> |
| <span class="sourceLineNo">175</span> while (!closed) {<a name="line.175"></a> |
| <span class="sourceLineNo">176</span> if (callsToWrite.isEmpty()) {<a name="line.176"></a> |
| <span class="sourceLineNo">177</span> // We should use another monitor object here for better performance since the read<a name="line.177"></a> |
| <span class="sourceLineNo">178</span> // thread also uses ConnectionImpl.this. But this makes the locking schema more<a name="line.178"></a> |
| <span class="sourceLineNo">179</span> // complicated, can do it later as an optimization.<a name="line.179"></a> |
| <span class="sourceLineNo">180</span> try {<a name="line.180"></a> |
| <span class="sourceLineNo">181</span> BlockingRpcConnection.this.wait();<a name="line.181"></a> |
| <span class="sourceLineNo">182</span> } catch (InterruptedException e) {<a name="line.182"></a> |
| <span class="sourceLineNo">183</span> // Restore interrupt status<a name="line.183"></a> |
| <span class="sourceLineNo">184</span> Thread.currentThread().interrupt();<a name="line.184"></a> |
| <span class="sourceLineNo">185</span> }<a name="line.185"></a> |
| <span class="sourceLineNo">186</span> // check if we need to quit, so continue the main loop instead of fallback.<a name="line.186"></a> |
| <span class="sourceLineNo">187</span> continue;<a name="line.187"></a> |
| <span class="sourceLineNo">188</span> }<a name="line.188"></a> |
| <span class="sourceLineNo">189</span> Call call = callsToWrite.poll();<a name="line.189"></a> |
| <span class="sourceLineNo">190</span> if (call.isDone()) {<a name="line.190"></a> |
| <span class="sourceLineNo">191</span> continue;<a name="line.191"></a> |
| <span class="sourceLineNo">192</span> }<a name="line.192"></a> |
| <span class="sourceLineNo">193</span> try (Scope scope = call.span.makeCurrent()) {<a name="line.193"></a> |
| <span class="sourceLineNo">194</span> writeRequest(call);<a name="line.194"></a> |
| <span class="sourceLineNo">195</span> } catch (IOException e) {<a name="line.195"></a> |
| <span class="sourceLineNo">196</span> // exception here means the call has not been added to the pendingCalls yet, so we need<a name="line.196"></a> |
| <span class="sourceLineNo">197</span> // to fail it by our own.<a name="line.197"></a> |
| <span class="sourceLineNo">198</span> LOG.debug("call write error for {}", call.toShortString());<a name="line.198"></a> |
| <span class="sourceLineNo">199</span> call.setException(e);<a name="line.199"></a> |
| <span class="sourceLineNo">200</span> closeConn(e);<a name="line.200"></a> |
| <span class="sourceLineNo">201</span> }<a name="line.201"></a> |
| <span class="sourceLineNo">202</span> }<a name="line.202"></a> |
| <span class="sourceLineNo">203</span> }<a name="line.203"></a> |
| <span class="sourceLineNo">204</span> }<a name="line.204"></a> |
| <span class="sourceLineNo">205</span><a name="line.205"></a> |
| <span class="sourceLineNo">206</span> /**<a name="line.206"></a> |
| <span class="sourceLineNo">207</span> * Cleans the call not yet sent when we finish.<a name="line.207"></a> |
| <span class="sourceLineNo">208</span> */<a name="line.208"></a> |
| <span class="sourceLineNo">209</span> public void cleanup(IOException e) {<a name="line.209"></a> |
| <span class="sourceLineNo">210</span> IOException ie =<a name="line.210"></a> |
| <span class="sourceLineNo">211</span> new ConnectionClosingException("Connection to " + remoteId.getAddress() + " is closing.");<a name="line.211"></a> |
| <span class="sourceLineNo">212</span> for (Call call : callsToWrite) {<a name="line.212"></a> |
| <span class="sourceLineNo">213</span> call.setException(ie);<a name="line.213"></a> |
| <span class="sourceLineNo">214</span> }<a name="line.214"></a> |
| <span class="sourceLineNo">215</span> callsToWrite.clear();<a name="line.215"></a> |
| <span class="sourceLineNo">216</span> }<a name="line.216"></a> |
| <span class="sourceLineNo">217</span> }<a name="line.217"></a> |
| <span class="sourceLineNo">218</span><a name="line.218"></a> |
| <span class="sourceLineNo">219</span> BlockingRpcConnection(BlockingRpcClient rpcClient, ConnectionId remoteId) throws IOException {<a name="line.219"></a> |
| <span class="sourceLineNo">220</span> super(rpcClient.conf, AbstractRpcClient.WHEEL_TIMER, remoteId, rpcClient.clusterId,<a name="line.220"></a> |
| <span class="sourceLineNo">221</span> rpcClient.userProvider.isHBaseSecurityEnabled(), rpcClient.codec, rpcClient.compressor,<a name="line.221"></a> |
| <span class="sourceLineNo">222</span> rpcClient.metrics);<a name="line.222"></a> |
| <span class="sourceLineNo">223</span> this.rpcClient = rpcClient;<a name="line.223"></a> |
| <span class="sourceLineNo">224</span> this.connectionHeaderPreamble = getConnectionHeaderPreamble();<a name="line.224"></a> |
| <span class="sourceLineNo">225</span> ConnectionHeader header = getConnectionHeader();<a name="line.225"></a> |
| <span class="sourceLineNo">226</span> ByteArrayOutputStream baos = new ByteArrayOutputStream(4 + header.getSerializedSize());<a name="line.226"></a> |
| <span class="sourceLineNo">227</span> DataOutputStream dos = new DataOutputStream(baos);<a name="line.227"></a> |
| <span class="sourceLineNo">228</span> dos.writeInt(header.getSerializedSize());<a name="line.228"></a> |
| <span class="sourceLineNo">229</span> header.writeTo(dos);<a name="line.229"></a> |
| <span class="sourceLineNo">230</span> assert baos.size() == 4 + header.getSerializedSize();<a name="line.230"></a> |
| <span class="sourceLineNo">231</span> this.connectionHeaderWithLength = baos.getBuffer();<a name="line.231"></a> |
| <span class="sourceLineNo">232</span><a name="line.232"></a> |
| <span class="sourceLineNo">233</span> UserGroupInformation ticket = remoteId.ticket.getUGI();<a name="line.233"></a> |
| <span class="sourceLineNo">234</span> this.threadName = "BRPC Connection (" + this.rpcClient.socketFactory.hashCode() + ") to "<a name="line.234"></a> |
| <span class="sourceLineNo">235</span> + remoteId.getAddress().toString()<a name="line.235"></a> |
| <span class="sourceLineNo">236</span> + ((ticket == null) ? " from an unknown user" : (" from " + ticket.getUserName()));<a name="line.236"></a> |
| <span class="sourceLineNo">237</span><a name="line.237"></a> |
| <span class="sourceLineNo">238</span> if (this.rpcClient.conf.getBoolean(BlockingRpcClient.SPECIFIC_WRITE_THREAD, false)) {<a name="line.238"></a> |
| <span class="sourceLineNo">239</span> callSender = new CallSender(threadName, this.rpcClient.conf);<a name="line.239"></a> |
| <span class="sourceLineNo">240</span> callSender.start();<a name="line.240"></a> |
| <span class="sourceLineNo">241</span> } else {<a name="line.241"></a> |
| <span class="sourceLineNo">242</span> callSender = null;<a name="line.242"></a> |
| <span class="sourceLineNo">243</span> }<a name="line.243"></a> |
| <span class="sourceLineNo">244</span> }<a name="line.244"></a> |
| <span class="sourceLineNo">245</span><a name="line.245"></a> |
| <span class="sourceLineNo">246</span> // protected for write UT.<a name="line.246"></a> |
| <span class="sourceLineNo">247</span> protected void setupConnection() throws IOException {<a name="line.247"></a> |
| <span class="sourceLineNo">248</span> short ioFailures = 0;<a name="line.248"></a> |
| <span class="sourceLineNo">249</span> short timeoutFailures = 0;<a name="line.249"></a> |
| <span class="sourceLineNo">250</span> while (true) {<a name="line.250"></a> |
| <span class="sourceLineNo">251</span> try {<a name="line.251"></a> |
| <span class="sourceLineNo">252</span> this.socket = this.rpcClient.socketFactory.createSocket();<a name="line.252"></a> |
| <span class="sourceLineNo">253</span> this.socket.setTcpNoDelay(this.rpcClient.isTcpNoDelay());<a name="line.253"></a> |
| <span class="sourceLineNo">254</span> this.socket.setKeepAlive(this.rpcClient.tcpKeepAlive);<a name="line.254"></a> |
| <span class="sourceLineNo">255</span> if (this.rpcClient.localAddr != null) {<a name="line.255"></a> |
| <span class="sourceLineNo">256</span> this.socket.bind(this.rpcClient.localAddr);<a name="line.256"></a> |
| <span class="sourceLineNo">257</span> }<a name="line.257"></a> |
| <span class="sourceLineNo">258</span> InetSocketAddress remoteAddr = getRemoteInetAddress(rpcClient.metrics);<a name="line.258"></a> |
| <span class="sourceLineNo">259</span> NetUtils.connect(this.socket, remoteAddr, this.rpcClient.connectTO);<a name="line.259"></a> |
| <span class="sourceLineNo">260</span> this.socket.setSoTimeout(this.rpcClient.readTO);<a name="line.260"></a> |
| <span class="sourceLineNo">261</span> return;<a name="line.261"></a> |
| <span class="sourceLineNo">262</span> } catch (SocketTimeoutException toe) {<a name="line.262"></a> |
| <span class="sourceLineNo">263</span> /*<a name="line.263"></a> |
| <span class="sourceLineNo">264</span> * The max number of retries is 45, which amounts to 20s*45 = 15 minutes retries.<a name="line.264"></a> |
| <span class="sourceLineNo">265</span> */<a name="line.265"></a> |
| <span class="sourceLineNo">266</span> if (LOG.isDebugEnabled()) {<a name="line.266"></a> |
| <span class="sourceLineNo">267</span> LOG.debug(<a name="line.267"></a> |
| <span class="sourceLineNo">268</span> "Received exception in connection setup.\n" + StringUtils.stringifyException(toe));<a name="line.268"></a> |
| <span class="sourceLineNo">269</span> }<a name="line.269"></a> |
| <span class="sourceLineNo">270</span> handleConnectionFailure(timeoutFailures++, this.rpcClient.maxRetries, toe);<a name="line.270"></a> |
| <span class="sourceLineNo">271</span> } catch (IOException ie) {<a name="line.271"></a> |
| <span class="sourceLineNo">272</span> if (LOG.isDebugEnabled()) {<a name="line.272"></a> |
| <span class="sourceLineNo">273</span> LOG.debug(<a name="line.273"></a> |
| <span class="sourceLineNo">274</span> "Received exception in connection setup.\n" + StringUtils.stringifyException(ie));<a name="line.274"></a> |
| <span class="sourceLineNo">275</span> }<a name="line.275"></a> |
| <span class="sourceLineNo">276</span> handleConnectionFailure(ioFailures++, this.rpcClient.maxRetries, ie);<a name="line.276"></a> |
| <span class="sourceLineNo">277</span> }<a name="line.277"></a> |
| <span class="sourceLineNo">278</span> }<a name="line.278"></a> |
| <span class="sourceLineNo">279</span> }<a name="line.279"></a> |
| <span class="sourceLineNo">280</span><a name="line.280"></a> |
| <span class="sourceLineNo">281</span> /**<a name="line.281"></a> |
| <span class="sourceLineNo">282</span> * Handle connection failures If the current number of retries is equal to the max number of<a name="line.282"></a> |
| <span class="sourceLineNo">283</span> * retries, stop retrying and throw the exception; Otherwise backoff N seconds and try connecting<a name="line.283"></a> |
| <span class="sourceLineNo">284</span> * again. This Method is only called from inside setupIOstreams(), which is synchronized. Hence<a name="line.284"></a> |
| <span class="sourceLineNo">285</span> * the sleep is synchronized; the locks will be retained.<a name="line.285"></a> |
| <span class="sourceLineNo">286</span> * @param curRetries current number of retries<a name="line.286"></a> |
| <span class="sourceLineNo">287</span> * @param maxRetries max number of retries allowed<a name="line.287"></a> |
| <span class="sourceLineNo">288</span> * @param ioe failure reason<a name="line.288"></a> |
| <span class="sourceLineNo">289</span> * @throws IOException if max number of retries is reached<a name="line.289"></a> |
| <span class="sourceLineNo">290</span> */<a name="line.290"></a> |
| <span class="sourceLineNo">291</span> private void handleConnectionFailure(int curRetries, int maxRetries, IOException ioe)<a name="line.291"></a> |
| <span class="sourceLineNo">292</span> throws IOException {<a name="line.292"></a> |
| <span class="sourceLineNo">293</span> closeSocket();<a name="line.293"></a> |
| <span class="sourceLineNo">294</span><a name="line.294"></a> |
| <span class="sourceLineNo">295</span> // throw the exception if the maximum number of retries is reached<a name="line.295"></a> |
| <span class="sourceLineNo">296</span> if (curRetries >= maxRetries || ExceptionUtil.isInterrupt(ioe)) {<a name="line.296"></a> |
| <span class="sourceLineNo">297</span> throw ioe;<a name="line.297"></a> |
| <span class="sourceLineNo">298</span> }<a name="line.298"></a> |
| <span class="sourceLineNo">299</span><a name="line.299"></a> |
| <span class="sourceLineNo">300</span> // otherwise back off and retry<a name="line.300"></a> |
| <span class="sourceLineNo">301</span> try {<a name="line.301"></a> |
| <span class="sourceLineNo">302</span> Thread.sleep(this.rpcClient.failureSleep);<a name="line.302"></a> |
| <span class="sourceLineNo">303</span> } catch (InterruptedException ie) {<a name="line.303"></a> |
| <span class="sourceLineNo">304</span> ExceptionUtil.rethrowIfInterrupt(ie);<a name="line.304"></a> |
| <span class="sourceLineNo">305</span> }<a name="line.305"></a> |
| <span class="sourceLineNo">306</span><a name="line.306"></a> |
| <span class="sourceLineNo">307</span> if (LOG.isInfoEnabled()) {<a name="line.307"></a> |
| <span class="sourceLineNo">308</span> LOG.info("Retrying connect to server: " + remoteId.getAddress() + " after sleeping "<a name="line.308"></a> |
| <span class="sourceLineNo">309</span> + this.rpcClient.failureSleep + "ms. Already tried " + curRetries + " time(s).");<a name="line.309"></a> |
| <span class="sourceLineNo">310</span> }<a name="line.310"></a> |
| <span class="sourceLineNo">311</span> }<a name="line.311"></a> |
| <span class="sourceLineNo">312</span><a name="line.312"></a> |
| <span class="sourceLineNo">313</span> /*<a name="line.313"></a> |
| <span class="sourceLineNo">314</span> * wait till someone signals us to start reading RPC response or it is idle too long, it is marked<a name="line.314"></a> |
| <span class="sourceLineNo">315</span> * as to be closed, or the client is marked as not running.<a name="line.315"></a> |
| <span class="sourceLineNo">316</span> * @return true if it is time to read a response; false otherwise.<a name="line.316"></a> |
| <span class="sourceLineNo">317</span> */<a name="line.317"></a> |
| <span class="sourceLineNo">318</span> private synchronized boolean waitForWork() {<a name="line.318"></a> |
| <span class="sourceLineNo">319</span> // beware of the concurrent access to the calls list: we can add calls, but as well<a name="line.319"></a> |
| <span class="sourceLineNo">320</span> // remove them.<a name="line.320"></a> |
| <span class="sourceLineNo">321</span> long waitUntil = EnvironmentEdgeManager.currentTime() + this.rpcClient.minIdleTimeBeforeClose;<a name="line.321"></a> |
| <span class="sourceLineNo">322</span> for (;;) {<a name="line.322"></a> |
| <span class="sourceLineNo">323</span> if (thread == null) {<a name="line.323"></a> |
| <span class="sourceLineNo">324</span> return false;<a name="line.324"></a> |
| <span class="sourceLineNo">325</span> }<a name="line.325"></a> |
| <span class="sourceLineNo">326</span> if (!calls.isEmpty()) {<a name="line.326"></a> |
| <span class="sourceLineNo">327</span> return true;<a name="line.327"></a> |
| <span class="sourceLineNo">328</span> }<a name="line.328"></a> |
| <span class="sourceLineNo">329</span> if (EnvironmentEdgeManager.currentTime() >= waitUntil) {<a name="line.329"></a> |
| <span class="sourceLineNo">330</span> closeConn(<a name="line.330"></a> |
| <span class="sourceLineNo">331</span> new IOException("idle connection closed with " + calls.size() + " pending request(s)"));<a name="line.331"></a> |
| <span class="sourceLineNo">332</span> return false;<a name="line.332"></a> |
| <span class="sourceLineNo">333</span> }<a name="line.333"></a> |
| <span class="sourceLineNo">334</span> try {<a name="line.334"></a> |
| <span class="sourceLineNo">335</span> wait(Math.min(this.rpcClient.minIdleTimeBeforeClose, 1000));<a name="line.335"></a> |
| <span class="sourceLineNo">336</span> } catch (InterruptedException e) {<a name="line.336"></a> |
| <span class="sourceLineNo">337</span> // Restore interrupt status<a name="line.337"></a> |
| <span class="sourceLineNo">338</span> Thread.currentThread().interrupt();<a name="line.338"></a> |
| <span class="sourceLineNo">339</span> }<a name="line.339"></a> |
| <span class="sourceLineNo">340</span> }<a name="line.340"></a> |
| <span class="sourceLineNo">341</span> }<a name="line.341"></a> |
| <span class="sourceLineNo">342</span><a name="line.342"></a> |
| <span class="sourceLineNo">343</span> @Override<a name="line.343"></a> |
| <span class="sourceLineNo">344</span> public void run() {<a name="line.344"></a> |
| <span class="sourceLineNo">345</span> if (LOG.isTraceEnabled()) {<a name="line.345"></a> |
| <span class="sourceLineNo">346</span> LOG.trace(threadName + ": starting");<a name="line.346"></a> |
| <span class="sourceLineNo">347</span> }<a name="line.347"></a> |
| <span class="sourceLineNo">348</span> while (waitForWork()) {<a name="line.348"></a> |
| <span class="sourceLineNo">349</span> readResponse();<a name="line.349"></a> |
| <span class="sourceLineNo">350</span> }<a name="line.350"></a> |
| <span class="sourceLineNo">351</span> if (LOG.isTraceEnabled()) {<a name="line.351"></a> |
| <span class="sourceLineNo">352</span> LOG.trace(threadName + ": stopped");<a name="line.352"></a> |
| <span class="sourceLineNo">353</span> }<a name="line.353"></a> |
| <span class="sourceLineNo">354</span> }<a name="line.354"></a> |
| <span class="sourceLineNo">355</span><a name="line.355"></a> |
| <span class="sourceLineNo">356</span> private void disposeSasl() {<a name="line.356"></a> |
| <span class="sourceLineNo">357</span> if (saslRpcClient != null) {<a name="line.357"></a> |
| <span class="sourceLineNo">358</span> saslRpcClient.dispose();<a name="line.358"></a> |
| <span class="sourceLineNo">359</span> saslRpcClient = null;<a name="line.359"></a> |
| <span class="sourceLineNo">360</span> }<a name="line.360"></a> |
| <span class="sourceLineNo">361</span> }<a name="line.361"></a> |
| <span class="sourceLineNo">362</span><a name="line.362"></a> |
| <span class="sourceLineNo">363</span> private boolean setupSaslConnection(final InputStream in2, final OutputStream out2)<a name="line.363"></a> |
| <span class="sourceLineNo">364</span> throws IOException {<a name="line.364"></a> |
| <span class="sourceLineNo">365</span> if (this.metrics != null) {<a name="line.365"></a> |
| <span class="sourceLineNo">366</span> this.metrics.incrNsLookups();<a name="line.366"></a> |
| <span class="sourceLineNo">367</span> }<a name="line.367"></a> |
| <span class="sourceLineNo">368</span> saslRpcClient = new HBaseSaslRpcClient(this.rpcClient.conf, provider, token,<a name="line.368"></a> |
| <span class="sourceLineNo">369</span> socket.getInetAddress(), securityInfo, this.rpcClient.fallbackAllowed,<a name="line.369"></a> |
| <span class="sourceLineNo">370</span> this.rpcClient.conf.get("hbase.rpc.protection",<a name="line.370"></a> |
| <span class="sourceLineNo">371</span> QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT)),<a name="line.371"></a> |
| <span class="sourceLineNo">372</span> this.rpcClient.conf.getBoolean(CRYPTO_AES_ENABLED_KEY, CRYPTO_AES_ENABLED_DEFAULT));<a name="line.372"></a> |
| <span class="sourceLineNo">373</span> return saslRpcClient.saslConnect(in2, out2);<a name="line.373"></a> |
| <span class="sourceLineNo">374</span> }<a name="line.374"></a> |
| <span class="sourceLineNo">375</span><a name="line.375"></a> |
| <span class="sourceLineNo">376</span> /**<a name="line.376"></a> |
| <span class="sourceLineNo">377</span> * If multiple clients with the same principal try to connect to the same server at the same time,<a name="line.377"></a> |
| <span class="sourceLineNo">378</span> * the server assumes a replay attack is in progress. This is a feature of kerberos. In order to<a name="line.378"></a> |
| <span class="sourceLineNo">379</span> * work around this, what is done is that the client backs off randomly and tries to initiate the<a name="line.379"></a> |
| <span class="sourceLineNo">380</span> * connection again. The other problem is to do with ticket expiry. To handle that, a relogin is<a name="line.380"></a> |
| <span class="sourceLineNo">381</span> * attempted.<a name="line.381"></a> |
| <span class="sourceLineNo">382</span> * <p><a name="line.382"></a> |
| <span class="sourceLineNo">383</span> * The retry logic is governed by the {@link SaslClientAuthenticationProvider#canRetry()} method.<a name="line.383"></a> |
| <span class="sourceLineNo">384</span> * Some providers have the ability to obtain new credentials and then re-attempt to authenticate<a name="line.384"></a> |
| <span class="sourceLineNo">385</span> * with HBase services. Other providers will continue to fail if they failed the first time -- for<a name="line.385"></a> |
| <span class="sourceLineNo">386</span> * those, we want to fail-fast.<a name="line.386"></a> |
| <span class="sourceLineNo">387</span> * </p><a name="line.387"></a> |
| <span class="sourceLineNo">388</span> */<a name="line.388"></a> |
| <span class="sourceLineNo">389</span> private void handleSaslConnectionFailure(final int currRetries, final int maxRetries,<a name="line.389"></a> |
| <span class="sourceLineNo">390</span> final Exception ex, final UserGroupInformation user) throws IOException, InterruptedException {<a name="line.390"></a> |
| <span class="sourceLineNo">391</span> closeSocket();<a name="line.391"></a> |
| <span class="sourceLineNo">392</span> user.doAs(new PrivilegedExceptionAction<Object>() {<a name="line.392"></a> |
| <span class="sourceLineNo">393</span> @Override<a name="line.393"></a> |
| <span class="sourceLineNo">394</span> public Object run() throws IOException, InterruptedException {<a name="line.394"></a> |
| <span class="sourceLineNo">395</span> // A provider which failed authentication, but doesn't have the ability to relogin with<a name="line.395"></a> |
| <span class="sourceLineNo">396</span> // some external system (e.g. username/password, the password either works or it doesn't)<a name="line.396"></a> |
| <span class="sourceLineNo">397</span> if (!provider.canRetry()) {<a name="line.397"></a> |
| <span class="sourceLineNo">398</span> LOG.warn("Exception encountered while connecting to the server " + remoteId.getAddress(),<a name="line.398"></a> |
| <span class="sourceLineNo">399</span> ex);<a name="line.399"></a> |
| <span class="sourceLineNo">400</span> if (ex instanceof RemoteException) {<a name="line.400"></a> |
| <span class="sourceLineNo">401</span> throw (RemoteException) ex;<a name="line.401"></a> |
| <span class="sourceLineNo">402</span> }<a name="line.402"></a> |
| <span class="sourceLineNo">403</span> if (ex instanceof SaslException) {<a name="line.403"></a> |
| <span class="sourceLineNo">404</span> String msg = "SASL authentication failed."<a name="line.404"></a> |
| <span class="sourceLineNo">405</span> + " The most likely cause is missing or invalid credentials.";<a name="line.405"></a> |
| <span class="sourceLineNo">406</span> throw new RuntimeException(msg, ex);<a name="line.406"></a> |
| <span class="sourceLineNo">407</span> }<a name="line.407"></a> |
| <span class="sourceLineNo">408</span> throw new IOException(ex);<a name="line.408"></a> |
| <span class="sourceLineNo">409</span> }<a name="line.409"></a> |
| <span class="sourceLineNo">410</span><a name="line.410"></a> |
| <span class="sourceLineNo">411</span> // Other providers, like kerberos, could request a new ticket from a keytab. Let<a name="line.411"></a> |
| <span class="sourceLineNo">412</span> // them try again.<a name="line.412"></a> |
| <span class="sourceLineNo">413</span> if (currRetries < maxRetries) {<a name="line.413"></a> |
| <span class="sourceLineNo">414</span> LOG.debug("Exception encountered while connecting to the server " + remoteId.getAddress(),<a name="line.414"></a> |
| <span class="sourceLineNo">415</span> ex);<a name="line.415"></a> |
| <span class="sourceLineNo">416</span><a name="line.416"></a> |
| <span class="sourceLineNo">417</span> // Invoke the provider to perform the relogin<a name="line.417"></a> |
| <span class="sourceLineNo">418</span> provider.relogin();<a name="line.418"></a> |
| <span class="sourceLineNo">419</span><a name="line.419"></a> |
| <span class="sourceLineNo">420</span> // Get rid of any old state on the SaslClient<a name="line.420"></a> |
| <span class="sourceLineNo">421</span> disposeSasl();<a name="line.421"></a> |
| <span class="sourceLineNo">422</span><a name="line.422"></a> |
| <span class="sourceLineNo">423</span> // have granularity of milliseconds<a name="line.423"></a> |
| <span class="sourceLineNo">424</span> // we are sleeping with the Connection lock held but since this<a name="line.424"></a> |
| <span class="sourceLineNo">425</span> // connection instance is being used for connecting to the server<a name="line.425"></a> |
| <span class="sourceLineNo">426</span> // in question, it is okay<a name="line.426"></a> |
| <span class="sourceLineNo">427</span> Thread.sleep(ThreadLocalRandom.current().nextInt(reloginMaxBackoff) + 1);<a name="line.427"></a> |
| <span class="sourceLineNo">428</span> return null;<a name="line.428"></a> |
| <span class="sourceLineNo">429</span> } else {<a name="line.429"></a> |
| <span class="sourceLineNo">430</span> String msg =<a name="line.430"></a> |
| <span class="sourceLineNo">431</span> "Failed to initiate connection for " + UserGroupInformation.getLoginUser().getUserName()<a name="line.431"></a> |
| <span class="sourceLineNo">432</span> + " to " + securityInfo.getServerPrincipal();<a name="line.432"></a> |
| <span class="sourceLineNo">433</span> throw new IOException(msg, ex);<a name="line.433"></a> |
| <span class="sourceLineNo">434</span> }<a name="line.434"></a> |
| <span class="sourceLineNo">435</span> }<a name="line.435"></a> |
| <span class="sourceLineNo">436</span> });<a name="line.436"></a> |
| <span class="sourceLineNo">437</span> }<a name="line.437"></a> |
| <span class="sourceLineNo">438</span><a name="line.438"></a> |
| <span class="sourceLineNo">439</span> private void setupIOstreams() throws IOException {<a name="line.439"></a> |
| <span class="sourceLineNo">440</span> if (socket != null) {<a name="line.440"></a> |
| <span class="sourceLineNo">441</span> // The connection is already available. Perfect.<a name="line.441"></a> |
| <span class="sourceLineNo">442</span> return;<a name="line.442"></a> |
| <span class="sourceLineNo">443</span> }<a name="line.443"></a> |
| <span class="sourceLineNo">444</span><a name="line.444"></a> |
| <span class="sourceLineNo">445</span> if (this.rpcClient.failedServers.isFailedServer(remoteId.getAddress())) {<a name="line.445"></a> |
| <span class="sourceLineNo">446</span> if (LOG.isDebugEnabled()) {<a name="line.446"></a> |
| <span class="sourceLineNo">447</span> LOG.debug("Not trying to connect to " + remoteId.getAddress()<a name="line.447"></a> |
| <span class="sourceLineNo">448</span> + " this server is in the failed servers list");<a name="line.448"></a> |
| <span class="sourceLineNo">449</span> }<a name="line.449"></a> |
| <span class="sourceLineNo">450</span> throw new FailedServerException(<a name="line.450"></a> |
| <span class="sourceLineNo">451</span> "This server is in the failed servers list: " + remoteId.getAddress());<a name="line.451"></a> |
| <span class="sourceLineNo">452</span> }<a name="line.452"></a> |
| <span class="sourceLineNo">453</span><a name="line.453"></a> |
| <span class="sourceLineNo">454</span> try {<a name="line.454"></a> |
| <span class="sourceLineNo">455</span> if (LOG.isDebugEnabled()) {<a name="line.455"></a> |
| <span class="sourceLineNo">456</span> LOG.debug("Connecting to " + remoteId.getAddress());<a name="line.456"></a> |
| <span class="sourceLineNo">457</span> }<a name="line.457"></a> |
| <span class="sourceLineNo">458</span><a name="line.458"></a> |
| <span class="sourceLineNo">459</span> short numRetries = 0;<a name="line.459"></a> |
| <span class="sourceLineNo">460</span> int reloginMaxRetries = this.rpcClient.conf.getInt("hbase.security.relogin.maxretries", 5);<a name="line.460"></a> |
| <span class="sourceLineNo">461</span> while (true) {<a name="line.461"></a> |
| <span class="sourceLineNo">462</span> setupConnection();<a name="line.462"></a> |
| <span class="sourceLineNo">463</span> InputStream inStream = NetUtils.getInputStream(socket);<a name="line.463"></a> |
| <span class="sourceLineNo">464</span> // This creates a socket with a write timeout. This timeout cannot be changed.<a name="line.464"></a> |
| <span class="sourceLineNo">465</span> OutputStream outStream = NetUtils.getOutputStream(socket, this.rpcClient.writeTO);<a name="line.465"></a> |
| <span class="sourceLineNo">466</span> // Write out the preamble -- MAGIC, version, and auth to use.<a name="line.466"></a> |
| <span class="sourceLineNo">467</span> writeConnectionHeaderPreamble(outStream);<a name="line.467"></a> |
| <span class="sourceLineNo">468</span> if (useSasl) {<a name="line.468"></a> |
| <span class="sourceLineNo">469</span> final InputStream in2 = inStream;<a name="line.469"></a> |
| <span class="sourceLineNo">470</span> final OutputStream out2 = outStream;<a name="line.470"></a> |
| <span class="sourceLineNo">471</span> UserGroupInformation ticket = provider.getRealUser(remoteId.ticket);<a name="line.471"></a> |
| <span class="sourceLineNo">472</span> boolean continueSasl;<a name="line.472"></a> |
| <span class="sourceLineNo">473</span> if (ticket == null) {<a name="line.473"></a> |
| <span class="sourceLineNo">474</span> throw new FatalConnectionException("ticket/user is null");<a name="line.474"></a> |
| <span class="sourceLineNo">475</span> }<a name="line.475"></a> |
| <span class="sourceLineNo">476</span> try {<a name="line.476"></a> |
| <span class="sourceLineNo">477</span> continueSasl = ticket.doAs(new PrivilegedExceptionAction<Boolean>() {<a name="line.477"></a> |
| <span class="sourceLineNo">478</span> @Override<a name="line.478"></a> |
| <span class="sourceLineNo">479</span> public Boolean run() throws IOException {<a name="line.479"></a> |
| <span class="sourceLineNo">480</span> return setupSaslConnection(in2, out2);<a name="line.480"></a> |
| <span class="sourceLineNo">481</span> }<a name="line.481"></a> |
| <span class="sourceLineNo">482</span> });<a name="line.482"></a> |
| <span class="sourceLineNo">483</span> } catch (Exception ex) {<a name="line.483"></a> |
| <span class="sourceLineNo">484</span> ExceptionUtil.rethrowIfInterrupt(ex);<a name="line.484"></a> |
| <span class="sourceLineNo">485</span> handleSaslConnectionFailure(numRetries++, reloginMaxRetries, ex, ticket);<a name="line.485"></a> |
| <span class="sourceLineNo">486</span> continue;<a name="line.486"></a> |
| <span class="sourceLineNo">487</span> }<a name="line.487"></a> |
| <span class="sourceLineNo">488</span> if (continueSasl) {<a name="line.488"></a> |
| <span class="sourceLineNo">489</span> // Sasl connect is successful. Let's set up Sasl i/o streams.<a name="line.489"></a> |
| <span class="sourceLineNo">490</span> inStream = saslRpcClient.getInputStream();<a name="line.490"></a> |
| <span class="sourceLineNo">491</span> outStream = saslRpcClient.getOutputStream();<a name="line.491"></a> |
| <span class="sourceLineNo">492</span> } else {<a name="line.492"></a> |
| <span class="sourceLineNo">493</span> // fall back to simple auth because server told us so.<a name="line.493"></a> |
| <span class="sourceLineNo">494</span> // do not change authMethod and useSasl here, we should start from secure when<a name="line.494"></a> |
| <span class="sourceLineNo">495</span> // reconnecting because regionserver may change its sasl config after restart.<a name="line.495"></a> |
| <span class="sourceLineNo">496</span> }<a name="line.496"></a> |
| <span class="sourceLineNo">497</span> }<a name="line.497"></a> |
| <span class="sourceLineNo">498</span> this.in = new DataInputStream(new BufferedInputStream(inStream));<a name="line.498"></a> |
| <span class="sourceLineNo">499</span> this.out = new DataOutputStream(new BufferedOutputStream(outStream));<a name="line.499"></a> |
| <span class="sourceLineNo">500</span> // Now write out the connection header<a name="line.500"></a> |
| <span class="sourceLineNo">501</span> writeConnectionHeader();<a name="line.501"></a> |
| <span class="sourceLineNo">502</span> // process the response from server for connection header if necessary<a name="line.502"></a> |
| <span class="sourceLineNo">503</span> processResponseForConnectionHeader();<a name="line.503"></a> |
| <span class="sourceLineNo">504</span><a name="line.504"></a> |
| <span class="sourceLineNo">505</span> break;<a name="line.505"></a> |
| <span class="sourceLineNo">506</span> }<a name="line.506"></a> |
| <span class="sourceLineNo">507</span> } catch (Throwable t) {<a name="line.507"></a> |
| <span class="sourceLineNo">508</span> closeSocket();<a name="line.508"></a> |
| <span class="sourceLineNo">509</span> IOException e = ExceptionUtil.asInterrupt(t);<a name="line.509"></a> |
| <span class="sourceLineNo">510</span> if (e == null) {<a name="line.510"></a> |
| <span class="sourceLineNo">511</span> this.rpcClient.failedServers.addToFailedServers(remoteId.getAddress(), t);<a name="line.511"></a> |
| <span class="sourceLineNo">512</span> if (t instanceof LinkageError) {<a name="line.512"></a> |
| <span class="sourceLineNo">513</span> // probably the hbase hadoop version does not match the running hadoop version<a name="line.513"></a> |
| <span class="sourceLineNo">514</span> e = new DoNotRetryIOException(t);<a name="line.514"></a> |
| <span class="sourceLineNo">515</span> } else if (t instanceof IOException) {<a name="line.515"></a> |
| <span class="sourceLineNo">516</span> e = (IOException) t;<a name="line.516"></a> |
| <span class="sourceLineNo">517</span> } else {<a name="line.517"></a> |
| <span class="sourceLineNo">518</span> e = new IOException("Could not set up IO Streams to " + remoteId.getAddress(), t);<a name="line.518"></a> |
| <span class="sourceLineNo">519</span> }<a name="line.519"></a> |
| <span class="sourceLineNo">520</span> }<a name="line.520"></a> |
| <span class="sourceLineNo">521</span> throw e;<a name="line.521"></a> |
| <span class="sourceLineNo">522</span> }<a name="line.522"></a> |
| <span class="sourceLineNo">523</span><a name="line.523"></a> |
| <span class="sourceLineNo">524</span> // start the receiver thread after the socket connection has been set up<a name="line.524"></a> |
| <span class="sourceLineNo">525</span> thread = new Thread(this, threadName);<a name="line.525"></a> |
| <span class="sourceLineNo">526</span> thread.setDaemon(true);<a name="line.526"></a> |
| <span class="sourceLineNo">527</span> thread.start();<a name="line.527"></a> |
| <span class="sourceLineNo">528</span> }<a name="line.528"></a> |
| <span class="sourceLineNo">529</span><a name="line.529"></a> |
| <span class="sourceLineNo">530</span> /**<a name="line.530"></a> |
| <span class="sourceLineNo">531</span> * Write the RPC header: {@code <MAGIC WORD -- 'HBas'> <ONEBYTE_VERSION> <ONEBYTE_AUTH_TYPE>}<a name="line.531"></a> |
| <span class="sourceLineNo">532</span> */<a name="line.532"></a> |
| <span class="sourceLineNo">533</span> private void writeConnectionHeaderPreamble(OutputStream out) throws IOException {<a name="line.533"></a> |
| <span class="sourceLineNo">534</span> out.write(connectionHeaderPreamble);<a name="line.534"></a> |
| <span class="sourceLineNo">535</span> out.flush();<a name="line.535"></a> |
| <span class="sourceLineNo">536</span> }<a name="line.536"></a> |
| <span class="sourceLineNo">537</span><a name="line.537"></a> |
| <span class="sourceLineNo">538</span> /**<a name="line.538"></a> |
| <span class="sourceLineNo">539</span> * Write the connection header.<a name="line.539"></a> |
| <span class="sourceLineNo">540</span> */<a name="line.540"></a> |
| <span class="sourceLineNo">541</span> private void writeConnectionHeader() throws IOException {<a name="line.541"></a> |
| <span class="sourceLineNo">542</span> boolean isCryptoAesEnable = false;<a name="line.542"></a> |
| <span class="sourceLineNo">543</span> // check if Crypto AES is enabled<a name="line.543"></a> |
| <span class="sourceLineNo">544</span> if (saslRpcClient != null) {<a name="line.544"></a> |
| <span class="sourceLineNo">545</span> boolean saslEncryptionEnabled = SaslUtil.QualityOfProtection.PRIVACY.getSaslQop()<a name="line.545"></a> |
| <span class="sourceLineNo">546</span> .equalsIgnoreCase(saslRpcClient.getSaslQOP());<a name="line.546"></a> |
| <span class="sourceLineNo">547</span> isCryptoAesEnable = saslEncryptionEnabled<a name="line.547"></a> |
| <span class="sourceLineNo">548</span> && conf.getBoolean(CRYPTO_AES_ENABLED_KEY, CRYPTO_AES_ENABLED_DEFAULT);<a name="line.548"></a> |
| <span class="sourceLineNo">549</span> }<a name="line.549"></a> |
| <span class="sourceLineNo">550</span><a name="line.550"></a> |
| <span class="sourceLineNo">551</span> // if Crypto AES is enabled, set transformation and negotiate with server<a name="line.551"></a> |
| <span class="sourceLineNo">552</span> if (isCryptoAesEnable) {<a name="line.552"></a> |
| <span class="sourceLineNo">553</span> waitingConnectionHeaderResponse = true;<a name="line.553"></a> |
| <span class="sourceLineNo">554</span> }<a name="line.554"></a> |
| <span class="sourceLineNo">555</span> this.out.write(connectionHeaderWithLength);<a name="line.555"></a> |
| <span class="sourceLineNo">556</span> this.out.flush();<a name="line.556"></a> |
| <span class="sourceLineNo">557</span> }<a name="line.557"></a> |
| <span class="sourceLineNo">558</span><a name="line.558"></a> |
| <span class="sourceLineNo">559</span> private void processResponseForConnectionHeader() throws IOException {<a name="line.559"></a> |
| <span class="sourceLineNo">560</span> // if no response excepted, return<a name="line.560"></a> |
| <span class="sourceLineNo">561</span> if (!waitingConnectionHeaderResponse) return;<a name="line.561"></a> |
| <span class="sourceLineNo">562</span> try {<a name="line.562"></a> |
| <span class="sourceLineNo">563</span> // read the ConnectionHeaderResponse from server<a name="line.563"></a> |
| <span class="sourceLineNo">564</span> int len = this.in.readInt();<a name="line.564"></a> |
| <span class="sourceLineNo">565</span> byte[] buff = new byte[len];<a name="line.565"></a> |
| <span class="sourceLineNo">566</span> int readSize = this.in.read(buff);<a name="line.566"></a> |
| <span class="sourceLineNo">567</span> if (LOG.isDebugEnabled()) {<a name="line.567"></a> |
| <span class="sourceLineNo">568</span> LOG.debug("Length of response for connection header:" + readSize);<a name="line.568"></a> |
| <span class="sourceLineNo">569</span> }<a name="line.569"></a> |
| <span class="sourceLineNo">570</span><a name="line.570"></a> |
| <span class="sourceLineNo">571</span> RPCProtos.ConnectionHeaderResponse connectionHeaderResponse =<a name="line.571"></a> |
| <span class="sourceLineNo">572</span> RPCProtos.ConnectionHeaderResponse.parseFrom(buff);<a name="line.572"></a> |
| <span class="sourceLineNo">573</span><a name="line.573"></a> |
| <span class="sourceLineNo">574</span> // Get the CryptoCipherMeta, update the HBaseSaslRpcClient for Crypto Cipher<a name="line.574"></a> |
| <span class="sourceLineNo">575</span> if (connectionHeaderResponse.hasCryptoCipherMeta()) {<a name="line.575"></a> |
| <span class="sourceLineNo">576</span> negotiateCryptoAes(connectionHeaderResponse.getCryptoCipherMeta());<a name="line.576"></a> |
| <span class="sourceLineNo">577</span> }<a name="line.577"></a> |
| <span class="sourceLineNo">578</span> waitingConnectionHeaderResponse = false;<a name="line.578"></a> |
| <span class="sourceLineNo">579</span> } catch (SocketTimeoutException ste) {<a name="line.579"></a> |
| <span class="sourceLineNo">580</span> LOG.error(HBaseMarkers.FATAL,<a name="line.580"></a> |
| <span class="sourceLineNo">581</span> "Can't get the connection header response for rpc timeout, "<a name="line.581"></a> |
| <span class="sourceLineNo">582</span> + "please check if server has the correct configuration to support the additional "<a name="line.582"></a> |
| <span class="sourceLineNo">583</span> + "function.",<a name="line.583"></a> |
| <span class="sourceLineNo">584</span> ste);<a name="line.584"></a> |
| <span class="sourceLineNo">585</span> // timeout when waiting the connection header response, ignore the additional function<a name="line.585"></a> |
| <span class="sourceLineNo">586</span> throw new IOException("Timeout while waiting connection header response", ste);<a name="line.586"></a> |
| <span class="sourceLineNo">587</span> }<a name="line.587"></a> |
| <span class="sourceLineNo">588</span> }<a name="line.588"></a> |
| <span class="sourceLineNo">589</span><a name="line.589"></a> |
| <span class="sourceLineNo">590</span> private void negotiateCryptoAes(RPCProtos.CryptoCipherMeta cryptoCipherMeta) throws IOException {<a name="line.590"></a> |
| <span class="sourceLineNo">591</span> // initialize the Crypto AES with CryptoCipherMeta<a name="line.591"></a> |
| <span class="sourceLineNo">592</span> saslRpcClient.initCryptoCipher(cryptoCipherMeta, this.rpcClient.conf);<a name="line.592"></a> |
| <span class="sourceLineNo">593</span> // reset the inputStream/outputStream for Crypto AES encryption<a name="line.593"></a> |
| <span class="sourceLineNo">594</span> this.in = new DataInputStream(new BufferedInputStream(saslRpcClient.getInputStream()));<a name="line.594"></a> |
| <span class="sourceLineNo">595</span> this.out = new DataOutputStream(new BufferedOutputStream(saslRpcClient.getOutputStream()));<a name="line.595"></a> |
| <span class="sourceLineNo">596</span> }<a name="line.596"></a> |
| <span class="sourceLineNo">597</span><a name="line.597"></a> |
| <span class="sourceLineNo">598</span> /**<a name="line.598"></a> |
| <span class="sourceLineNo">599</span> * Initiates a call by sending the parameter to the remote server. Note: this is not called from<a name="line.599"></a> |
| <span class="sourceLineNo">600</span> * the Connection thread, but by other threads.<a name="line.600"></a> |
| <span class="sourceLineNo">601</span> * @see #readResponse()<a name="line.601"></a> |
| <span class="sourceLineNo">602</span> */<a name="line.602"></a> |
| <span class="sourceLineNo">603</span> private void writeRequest(Call call) throws IOException {<a name="line.603"></a> |
| <span class="sourceLineNo">604</span> ByteBuf cellBlock = null;<a name="line.604"></a> |
| <span class="sourceLineNo">605</span> try {<a name="line.605"></a> |
| <span class="sourceLineNo">606</span> cellBlock = this.rpcClient.cellBlockBuilder.buildCellBlock(this.codec, this.compressor,<a name="line.606"></a> |
| <span class="sourceLineNo">607</span> call.cells, PooledByteBufAllocator.DEFAULT);<a name="line.607"></a> |
| <span class="sourceLineNo">608</span> CellBlockMeta cellBlockMeta;<a name="line.608"></a> |
| <span class="sourceLineNo">609</span> if (cellBlock != null) {<a name="line.609"></a> |
| <span class="sourceLineNo">610</span> cellBlockMeta = CellBlockMeta.newBuilder().setLength(cellBlock.readableBytes()).build();<a name="line.610"></a> |
| <span class="sourceLineNo">611</span> } else {<a name="line.611"></a> |
| <span class="sourceLineNo">612</span> cellBlockMeta = null;<a name="line.612"></a> |
| <span class="sourceLineNo">613</span> }<a name="line.613"></a> |
| <span class="sourceLineNo">614</span> RequestHeader requestHeader = buildRequestHeader(call, cellBlockMeta);<a name="line.614"></a> |
| <span class="sourceLineNo">615</span><a name="line.615"></a> |
| <span class="sourceLineNo">616</span> setupIOstreams();<a name="line.616"></a> |
| <span class="sourceLineNo">617</span><a name="line.617"></a> |
| <span class="sourceLineNo">618</span> // Now we're going to write the call. We take the lock, then check that the connection<a name="line.618"></a> |
| <span class="sourceLineNo">619</span> // is still valid, and, if so we do the write to the socket. If the write fails, we don't<a name="line.619"></a> |
| <span class="sourceLineNo">620</span> // know where we stand, we have to close the connection.<a name="line.620"></a> |
| <span class="sourceLineNo">621</span> if (Thread.interrupted()) {<a name="line.621"></a> |
| <span class="sourceLineNo">622</span> throw new InterruptedIOException();<a name="line.622"></a> |
| <span class="sourceLineNo">623</span> }<a name="line.623"></a> |
| <span class="sourceLineNo">624</span><a name="line.624"></a> |
| <span class="sourceLineNo">625</span> calls.put(call.id, call); // We put first as we don't want the connection to become idle.<a name="line.625"></a> |
| <span class="sourceLineNo">626</span> // from here, we do not throw any exception to upper layer as the call has been tracked in<a name="line.626"></a> |
| <span class="sourceLineNo">627</span> // the pending calls map.<a name="line.627"></a> |
| <span class="sourceLineNo">628</span> try {<a name="line.628"></a> |
| <span class="sourceLineNo">629</span> call.callStats.setRequestSizeBytes(write(this.out, requestHeader, call.param, cellBlock));<a name="line.629"></a> |
| <span class="sourceLineNo">630</span> } catch (Throwable t) {<a name="line.630"></a> |
| <span class="sourceLineNo">631</span> if (LOG.isTraceEnabled()) {<a name="line.631"></a> |
| <span class="sourceLineNo">632</span> LOG.trace("Error while writing {}", call.toShortString());<a name="line.632"></a> |
| <span class="sourceLineNo">633</span> }<a name="line.633"></a> |
| <span class="sourceLineNo">634</span> IOException e = IPCUtil.toIOE(t);<a name="line.634"></a> |
| <span class="sourceLineNo">635</span> closeConn(e);<a name="line.635"></a> |
| <span class="sourceLineNo">636</span> return;<a name="line.636"></a> |
| <span class="sourceLineNo">637</span> }<a name="line.637"></a> |
| <span class="sourceLineNo">638</span> } finally {<a name="line.638"></a> |
| <span class="sourceLineNo">639</span> if (cellBlock != null) {<a name="line.639"></a> |
| <span class="sourceLineNo">640</span> cellBlock.release();<a name="line.640"></a> |
| <span class="sourceLineNo">641</span> }<a name="line.641"></a> |
| <span class="sourceLineNo">642</span> }<a name="line.642"></a> |
| <span class="sourceLineNo">643</span> notifyAll();<a name="line.643"></a> |
| <span class="sourceLineNo">644</span> }<a name="line.644"></a> |
| <span class="sourceLineNo">645</span><a name="line.645"></a> |
| <span class="sourceLineNo">646</span> /*<a name="line.646"></a> |
| <span class="sourceLineNo">647</span> * Receive a response. Because only one receiver, so no synchronization on in.<a name="line.647"></a> |
| <span class="sourceLineNo">648</span> */<a name="line.648"></a> |
| <span class="sourceLineNo">649</span> private void readResponse() {<a name="line.649"></a> |
| <span class="sourceLineNo">650</span> Call call = null;<a name="line.650"></a> |
| <span class="sourceLineNo">651</span> boolean expectedCall = false;<a name="line.651"></a> |
| <span class="sourceLineNo">652</span> try {<a name="line.652"></a> |
| <span class="sourceLineNo">653</span> // See HBaseServer.Call.setResponse for where we write out the response.<a name="line.653"></a> |
| <span class="sourceLineNo">654</span> // Total size of the response. Unused. But have to read it in anyways.<a name="line.654"></a> |
| <span class="sourceLineNo">655</span> int totalSize = in.readInt();<a name="line.655"></a> |
| <span class="sourceLineNo">656</span><a name="line.656"></a> |
| <span class="sourceLineNo">657</span> // Read the header<a name="line.657"></a> |
| <span class="sourceLineNo">658</span> ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in);<a name="line.658"></a> |
| <span class="sourceLineNo">659</span> int id = responseHeader.getCallId();<a name="line.659"></a> |
| <span class="sourceLineNo">660</span> call = calls.remove(id); // call.done have to be set before leaving this method<a name="line.660"></a> |
| <span class="sourceLineNo">661</span> expectedCall = (call != null && !call.isDone());<a name="line.661"></a> |
| <span class="sourceLineNo">662</span> if (!expectedCall) {<a name="line.662"></a> |
| <span class="sourceLineNo">663</span> // So we got a response for which we have no corresponding 'call' here on the client-side.<a name="line.663"></a> |
| <span class="sourceLineNo">664</span> // We probably timed out waiting, cleaned up all references, and now the server decides<a name="line.664"></a> |
| <span class="sourceLineNo">665</span> // to return a response. There is nothing we can do w/ the response at this stage. Clean<a name="line.665"></a> |
| <span class="sourceLineNo">666</span> // out the wire of the response so its out of the way and we can get other responses on<a name="line.666"></a> |
| <span class="sourceLineNo">667</span> // this connection.<a name="line.667"></a> |
| <span class="sourceLineNo">668</span> int readSoFar = getTotalSizeWhenWrittenDelimited(responseHeader);<a name="line.668"></a> |
| <span class="sourceLineNo">669</span> int whatIsLeftToRead = totalSize - readSoFar;<a name="line.669"></a> |
| <span class="sourceLineNo">670</span> IOUtils.skipFully(in, whatIsLeftToRead);<a name="line.670"></a> |
| <span class="sourceLineNo">671</span> if (call != null) {<a name="line.671"></a> |
| <span class="sourceLineNo">672</span> call.callStats.setResponseSizeBytes(totalSize);<a name="line.672"></a> |
| <span class="sourceLineNo">673</span> call.callStats<a name="line.673"></a> |
| <span class="sourceLineNo">674</span> .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime());<a name="line.674"></a> |
| <span class="sourceLineNo">675</span> }<a name="line.675"></a> |
| <span class="sourceLineNo">676</span> return;<a name="line.676"></a> |
| <span class="sourceLineNo">677</span> }<a name="line.677"></a> |
| <span class="sourceLineNo">678</span> if (responseHeader.hasException()) {<a name="line.678"></a> |
| <span class="sourceLineNo">679</span> ExceptionResponse exceptionResponse = responseHeader.getException();<a name="line.679"></a> |
| <span class="sourceLineNo">680</span> RemoteException re = createRemoteException(exceptionResponse);<a name="line.680"></a> |
| <span class="sourceLineNo">681</span> call.setException(re);<a name="line.681"></a> |
| <span class="sourceLineNo">682</span> call.callStats.setResponseSizeBytes(totalSize);<a name="line.682"></a> |
| <span class="sourceLineNo">683</span> call.callStats<a name="line.683"></a> |
| <span class="sourceLineNo">684</span> .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime());<a name="line.684"></a> |
| <span class="sourceLineNo">685</span> if (isFatalConnectionException(exceptionResponse)) {<a name="line.685"></a> |
| <span class="sourceLineNo">686</span> synchronized (this) {<a name="line.686"></a> |
| <span class="sourceLineNo">687</span> closeConn(re);<a name="line.687"></a> |
| <span class="sourceLineNo">688</span> }<a name="line.688"></a> |
| <span class="sourceLineNo">689</span> }<a name="line.689"></a> |
| <span class="sourceLineNo">690</span> } else {<a name="line.690"></a> |
| <span class="sourceLineNo">691</span> Message value = null;<a name="line.691"></a> |
| <span class="sourceLineNo">692</span> if (call.responseDefaultType != null) {<a name="line.692"></a> |
| <span class="sourceLineNo">693</span> Message.Builder builder = call.responseDefaultType.newBuilderForType();<a name="line.693"></a> |
| <span class="sourceLineNo">694</span> ProtobufUtil.mergeDelimitedFrom(builder, in);<a name="line.694"></a> |
| <span class="sourceLineNo">695</span> value = builder.build();<a name="line.695"></a> |
| <span class="sourceLineNo">696</span> }<a name="line.696"></a> |
| <span class="sourceLineNo">697</span> CellScanner cellBlockScanner = null;<a name="line.697"></a> |
| <span class="sourceLineNo">698</span> if (responseHeader.hasCellBlockMeta()) {<a name="line.698"></a> |
| <span class="sourceLineNo">699</span> int size = responseHeader.getCellBlockMeta().getLength();<a name="line.699"></a> |
| <span class="sourceLineNo">700</span> byte[] cellBlock = new byte[size];<a name="line.700"></a> |
| <span class="sourceLineNo">701</span> IOUtils.readFully(this.in, cellBlock, 0, cellBlock.length);<a name="line.701"></a> |
| <span class="sourceLineNo">702</span> cellBlockScanner = this.rpcClient.cellBlockBuilder.createCellScanner(this.codec,<a name="line.702"></a> |
| <span class="sourceLineNo">703</span> this.compressor, cellBlock);<a name="line.703"></a> |
| <span class="sourceLineNo">704</span> }<a name="line.704"></a> |
| <span class="sourceLineNo">705</span> call.setResponse(value, cellBlockScanner);<a name="line.705"></a> |
| <span class="sourceLineNo">706</span> call.callStats.setResponseSizeBytes(totalSize);<a name="line.706"></a> |
| <span class="sourceLineNo">707</span> call.callStats<a name="line.707"></a> |
| <span class="sourceLineNo">708</span> .setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.callStats.getStartTime());<a name="line.708"></a> |
| <span class="sourceLineNo">709</span> }<a name="line.709"></a> |
| <span class="sourceLineNo">710</span> } catch (IOException e) {<a name="line.710"></a> |
| <span class="sourceLineNo">711</span> if (expectedCall) {<a name="line.711"></a> |
| <span class="sourceLineNo">712</span> call.setException(e);<a name="line.712"></a> |
| <span class="sourceLineNo">713</span> }<a name="line.713"></a> |
| <span class="sourceLineNo">714</span> if (e instanceof SocketTimeoutException) {<a name="line.714"></a> |
| <span class="sourceLineNo">715</span> // Clean up open calls but don't treat this as a fatal condition,<a name="line.715"></a> |
| <span class="sourceLineNo">716</span> // since we expect certain responses to not make it by the specified<a name="line.716"></a> |
| <span class="sourceLineNo">717</span> // {@link ConnectionId#rpcTimeout}.<a name="line.717"></a> |
| <span class="sourceLineNo">718</span> if (LOG.isTraceEnabled()) {<a name="line.718"></a> |
| <span class="sourceLineNo">719</span> LOG.trace("ignored", e);<a name="line.719"></a> |
| <span class="sourceLineNo">720</span> }<a name="line.720"></a> |
| <span class="sourceLineNo">721</span> } else {<a name="line.721"></a> |
| <span class="sourceLineNo">722</span> synchronized (this) {<a name="line.722"></a> |
| <span class="sourceLineNo">723</span> closeConn(e);<a name="line.723"></a> |
| <span class="sourceLineNo">724</span> }<a name="line.724"></a> |
| <span class="sourceLineNo">725</span> }<a name="line.725"></a> |
| <span class="sourceLineNo">726</span> }<a name="line.726"></a> |
| <span class="sourceLineNo">727</span> }<a name="line.727"></a> |
| <span class="sourceLineNo">728</span><a name="line.728"></a> |
| <span class="sourceLineNo">729</span> @Override<a name="line.729"></a> |
| <span class="sourceLineNo">730</span> protected synchronized void callTimeout(Call call) {<a name="line.730"></a> |
| <span class="sourceLineNo">731</span> // call sender<a name="line.731"></a> |
| <span class="sourceLineNo">732</span> calls.remove(call.id);<a name="line.732"></a> |
| <span class="sourceLineNo">733</span> }<a name="line.733"></a> |
| <span class="sourceLineNo">734</span><a name="line.734"></a> |
| <span class="sourceLineNo">735</span> // just close socket input and output.<a name="line.735"></a> |
| <span class="sourceLineNo">736</span> private void closeSocket() {<a name="line.736"></a> |
| <span class="sourceLineNo">737</span> IOUtils.closeStream(out);<a name="line.737"></a> |
| <span class="sourceLineNo">738</span> IOUtils.closeStream(in);<a name="line.738"></a> |
| <span class="sourceLineNo">739</span> IOUtils.closeSocket(socket);<a name="line.739"></a> |
| <span class="sourceLineNo">740</span> out = null;<a name="line.740"></a> |
| <span class="sourceLineNo">741</span> in = null;<a name="line.741"></a> |
| <span class="sourceLineNo">742</span> socket = null;<a name="line.742"></a> |
| <span class="sourceLineNo">743</span> }<a name="line.743"></a> |
| <span class="sourceLineNo">744</span><a name="line.744"></a> |
| <span class="sourceLineNo">745</span> // close socket, reader, and clean up all pending calls.<a name="line.745"></a> |
| <span class="sourceLineNo">746</span> private void closeConn(IOException e) {<a name="line.746"></a> |
| <span class="sourceLineNo">747</span> if (thread == null) {<a name="line.747"></a> |
| <span class="sourceLineNo">748</span> return;<a name="line.748"></a> |
| <span class="sourceLineNo">749</span> }<a name="line.749"></a> |
| <span class="sourceLineNo">750</span> thread.interrupt();<a name="line.750"></a> |
| <span class="sourceLineNo">751</span> thread = null;<a name="line.751"></a> |
| <span class="sourceLineNo">752</span> closeSocket();<a name="line.752"></a> |
| <span class="sourceLineNo">753</span> if (callSender != null) {<a name="line.753"></a> |
| <span class="sourceLineNo">754</span> callSender.cleanup(e);<a name="line.754"></a> |
| <span class="sourceLineNo">755</span> }<a name="line.755"></a> |
| <span class="sourceLineNo">756</span> for (Call call : calls.values()) {<a name="line.756"></a> |
| <span class="sourceLineNo">757</span> call.setException(e);<a name="line.757"></a> |
| <span class="sourceLineNo">758</span> }<a name="line.758"></a> |
| <span class="sourceLineNo">759</span> calls.clear();<a name="line.759"></a> |
| <span class="sourceLineNo">760</span> }<a name="line.760"></a> |
| <span class="sourceLineNo">761</span><a name="line.761"></a> |
| <span class="sourceLineNo">762</span> // release all resources, the connection will not be used any more.<a name="line.762"></a> |
| <span class="sourceLineNo">763</span> @Override<a name="line.763"></a> |
| <span class="sourceLineNo">764</span> public synchronized void shutdown() {<a name="line.764"></a> |
| <span class="sourceLineNo">765</span> closed = true;<a name="line.765"></a> |
| <span class="sourceLineNo">766</span> if (callSender != null) {<a name="line.766"></a> |
| <span class="sourceLineNo">767</span> callSender.interrupt();<a name="line.767"></a> |
| <span class="sourceLineNo">768</span> }<a name="line.768"></a> |
| <span class="sourceLineNo">769</span> closeConn(new IOException("connection to " + remoteId.getAddress() + " closed"));<a name="line.769"></a> |
| <span class="sourceLineNo">770</span> }<a name="line.770"></a> |
| <span class="sourceLineNo">771</span><a name="line.771"></a> |
| <span class="sourceLineNo">772</span> @Override<a name="line.772"></a> |
| <span class="sourceLineNo">773</span> public void cleanupConnection() {<a name="line.773"></a> |
| <span class="sourceLineNo">774</span> // do nothing<a name="line.774"></a> |
| <span class="sourceLineNo">775</span> }<a name="line.775"></a> |
| <span class="sourceLineNo">776</span><a name="line.776"></a> |
| <span class="sourceLineNo">777</span> @Override<a name="line.777"></a> |
| <span class="sourceLineNo">778</span> public synchronized void sendRequest(final Call call, HBaseRpcController pcrc)<a name="line.778"></a> |
| <span class="sourceLineNo">779</span> throws IOException {<a name="line.779"></a> |
| <span class="sourceLineNo">780</span> pcrc.notifyOnCancel(new RpcCallback<Object>() {<a name="line.780"></a> |
| <span class="sourceLineNo">781</span><a name="line.781"></a> |
| <span class="sourceLineNo">782</span> @Override<a name="line.782"></a> |
| <span class="sourceLineNo">783</span> public void run(Object parameter) {<a name="line.783"></a> |
| <span class="sourceLineNo">784</span> setCancelled(call);<a name="line.784"></a> |
| <span class="sourceLineNo">785</span> synchronized (BlockingRpcConnection.this) {<a name="line.785"></a> |
| <span class="sourceLineNo">786</span> if (callSender != null) {<a name="line.786"></a> |
| <span class="sourceLineNo">787</span> callSender.remove(call);<a name="line.787"></a> |
| <span class="sourceLineNo">788</span> } else {<a name="line.788"></a> |
| <span class="sourceLineNo">789</span> calls.remove(call.id);<a name="line.789"></a> |
| <span class="sourceLineNo">790</span> }<a name="line.790"></a> |
| <span class="sourceLineNo">791</span> }<a name="line.791"></a> |
| <span class="sourceLineNo">792</span> }<a name="line.792"></a> |
| <span class="sourceLineNo">793</span> }, new CancellationCallback() {<a name="line.793"></a> |
| <span class="sourceLineNo">794</span><a name="line.794"></a> |
| <span class="sourceLineNo">795</span> @Override<a name="line.795"></a> |
| <span class="sourceLineNo">796</span> public void run(boolean cancelled) throws IOException {<a name="line.796"></a> |
| <span class="sourceLineNo">797</span> if (cancelled) {<a name="line.797"></a> |
| <span class="sourceLineNo">798</span> setCancelled(call);<a name="line.798"></a> |
| <span class="sourceLineNo">799</span> return;<a name="line.799"></a> |
| <span class="sourceLineNo">800</span> }<a name="line.800"></a> |
| <span class="sourceLineNo">801</span> scheduleTimeoutTask(call);<a name="line.801"></a> |
| <span class="sourceLineNo">802</span> if (callSender != null) {<a name="line.802"></a> |
| <span class="sourceLineNo">803</span> callSender.sendCall(call);<a name="line.803"></a> |
| <span class="sourceLineNo">804</span> } else {<a name="line.804"></a> |
| <span class="sourceLineNo">805</span> // this is in the same thread with the caller so do not need to attach the trace context<a name="line.805"></a> |
| <span class="sourceLineNo">806</span> // again.<a name="line.806"></a> |
| <span class="sourceLineNo">807</span> writeRequest(call);<a name="line.807"></a> |
| <span class="sourceLineNo">808</span> }<a name="line.808"></a> |
| <span class="sourceLineNo">809</span> }<a name="line.809"></a> |
| <span class="sourceLineNo">810</span> });<a name="line.810"></a> |
| <span class="sourceLineNo">811</span> }<a name="line.811"></a> |
| <span class="sourceLineNo">812</span><a name="line.812"></a> |
| <span class="sourceLineNo">813</span> @Override<a name="line.813"></a> |
| <span class="sourceLineNo">814</span> public synchronized boolean isActive() {<a name="line.814"></a> |
| <span class="sourceLineNo">815</span> return thread != null;<a name="line.815"></a> |
| <span class="sourceLineNo">816</span> }<a name="line.816"></a> |
| <span class="sourceLineNo">817</span>}<a name="line.817"></a> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| </pre> |
| </div> |
| </body> |
| </html> |