| <!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.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION;<a name="line.20"></a> |
| <span class="sourceLineNo">021</span><a name="line.21"></a> |
| <span class="sourceLineNo">022</span>import java.io.IOException;<a name="line.22"></a> |
| <span class="sourceLineNo">023</span>import java.net.InetAddress;<a name="line.23"></a> |
| <span class="sourceLineNo">024</span>import java.net.InetSocketAddress;<a name="line.24"></a> |
| <span class="sourceLineNo">025</span>import java.nio.ByteBuffer;<a name="line.25"></a> |
| <span class="sourceLineNo">026</span>import java.nio.channels.ReadableByteChannel;<a name="line.26"></a> |
| <span class="sourceLineNo">027</span>import java.nio.channels.WritableByteChannel;<a name="line.27"></a> |
| <span class="sourceLineNo">028</span>import java.util.Collections;<a name="line.28"></a> |
| <span class="sourceLineNo">029</span>import java.util.HashMap;<a name="line.29"></a> |
| <span class="sourceLineNo">030</span>import java.util.List;<a name="line.30"></a> |
| <span class="sourceLineNo">031</span>import java.util.Locale;<a name="line.31"></a> |
| <span class="sourceLineNo">032</span>import java.util.Map;<a name="line.32"></a> |
| <span class="sourceLineNo">033</span>import java.util.Optional;<a name="line.33"></a> |
| <span class="sourceLineNo">034</span>import java.util.concurrent.atomic.LongAdder;<a name="line.34"></a> |
| <span class="sourceLineNo">035</span>import org.apache.commons.lang3.StringUtils;<a name="line.35"></a> |
| <span class="sourceLineNo">036</span>import org.apache.hadoop.conf.Configuration;<a name="line.36"></a> |
| <span class="sourceLineNo">037</span>import org.apache.hadoop.hbase.CallQueueTooBigException;<a name="line.37"></a> |
| <span class="sourceLineNo">038</span>import org.apache.hadoop.hbase.CellScanner;<a name="line.38"></a> |
| <span class="sourceLineNo">039</span>import org.apache.hadoop.hbase.DoNotRetryIOException;<a name="line.39"></a> |
| <span class="sourceLineNo">040</span>import org.apache.hadoop.hbase.HConstants;<a name="line.40"></a> |
| <span class="sourceLineNo">041</span>import org.apache.hadoop.hbase.Server;<a name="line.41"></a> |
| <span class="sourceLineNo">042</span>import org.apache.hadoop.hbase.conf.ConfigurationObserver;<a name="line.42"></a> |
| <span class="sourceLineNo">043</span>import org.apache.hadoop.hbase.io.ByteBuffAllocator;<a name="line.43"></a> |
| <span class="sourceLineNo">044</span>import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;<a name="line.44"></a> |
| <span class="sourceLineNo">045</span>import org.apache.hadoop.hbase.monitoring.TaskMonitor;<a name="line.45"></a> |
| <span class="sourceLineNo">046</span>import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;<a name="line.46"></a> |
| <span class="sourceLineNo">047</span>import org.apache.hadoop.hbase.namequeues.RpcLogDetails;<a name="line.47"></a> |
| <span class="sourceLineNo">048</span>import org.apache.hadoop.hbase.regionserver.RSRpcServices;<a name="line.48"></a> |
| <span class="sourceLineNo">049</span>import org.apache.hadoop.hbase.security.HBasePolicyProvider;<a name="line.49"></a> |
| <span class="sourceLineNo">050</span>import org.apache.hadoop.hbase.security.SaslUtil;<a name="line.50"></a> |
| <span class="sourceLineNo">051</span>import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;<a name="line.51"></a> |
| <span class="sourceLineNo">052</span>import org.apache.hadoop.hbase.security.User;<a name="line.52"></a> |
| <span class="sourceLineNo">053</span>import org.apache.hadoop.hbase.security.UserProvider;<a name="line.53"></a> |
| <span class="sourceLineNo">054</span>import org.apache.hadoop.hbase.security.token.AuthenticationTokenSecretManager;<a name="line.54"></a> |
| <span class="sourceLineNo">055</span>import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;<a name="line.55"></a> |
| <span class="sourceLineNo">056</span>import org.apache.hadoop.hbase.util.GsonUtil;<a name="line.56"></a> |
| <span class="sourceLineNo">057</span>import org.apache.hadoop.hbase.util.Pair;<a name="line.57"></a> |
| <span class="sourceLineNo">058</span>import org.apache.hadoop.security.UserGroupInformation;<a name="line.58"></a> |
| <span class="sourceLineNo">059</span>import org.apache.hadoop.security.authorize.AuthorizationException;<a name="line.59"></a> |
| <span class="sourceLineNo">060</span>import org.apache.hadoop.security.authorize.PolicyProvider;<a name="line.60"></a> |
| <span class="sourceLineNo">061</span>import org.apache.hadoop.security.authorize.ProxyUsers;<a name="line.61"></a> |
| <span class="sourceLineNo">062</span>import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;<a name="line.62"></a> |
| <span class="sourceLineNo">063</span>import org.apache.hadoop.security.token.SecretManager;<a name="line.63"></a> |
| <span class="sourceLineNo">064</span>import org.apache.hadoop.security.token.TokenIdentifier;<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.gson.Gson;<a name="line.69"></a> |
| <span class="sourceLineNo">070</span>import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;<a name="line.70"></a> |
| <span class="sourceLineNo">071</span>import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;<a name="line.71"></a> |
| <span class="sourceLineNo">072</span>import org.apache.hbase.thirdparty.com.google.protobuf.Message;<a name="line.72"></a> |
| <span class="sourceLineNo">073</span>import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;<a name="line.73"></a> |
| <span class="sourceLineNo">074</span>import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;<a name="line.74"></a> |
| <span class="sourceLineNo">075</span><a name="line.75"></a> |
| <span class="sourceLineNo">076</span>import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;<a name="line.76"></a> |
| <span class="sourceLineNo">077</span>import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;<a name="line.77"></a> |
| <span class="sourceLineNo">078</span>import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader;<a name="line.78"></a> |
| <span class="sourceLineNo">079</span><a name="line.79"></a> |
| <span class="sourceLineNo">080</span>/**<a name="line.80"></a> |
| <span class="sourceLineNo">081</span> * An RPC server that hosts protobuf described Services.<a name="line.81"></a> |
| <span class="sourceLineNo">082</span> */<a name="line.82"></a> |
| <span class="sourceLineNo">083</span>@InterfaceAudience.Private<a name="line.83"></a> |
| <span class="sourceLineNo">084</span>public abstract class RpcServer implements RpcServerInterface, ConfigurationObserver {<a name="line.84"></a> |
| <span class="sourceLineNo">085</span> // LOG is being used in CallRunner and the log level is being changed in tests<a name="line.85"></a> |
| <span class="sourceLineNo">086</span> public static final Logger LOG = LoggerFactory.getLogger(RpcServer.class);<a name="line.86"></a> |
| <span class="sourceLineNo">087</span> protected static final CallQueueTooBigException CALL_QUEUE_TOO_BIG_EXCEPTION =<a name="line.87"></a> |
| <span class="sourceLineNo">088</span> new CallQueueTooBigException();<a name="line.88"></a> |
| <span class="sourceLineNo">089</span><a name="line.89"></a> |
| <span class="sourceLineNo">090</span> private static final String MULTI_GETS = "multi.gets";<a name="line.90"></a> |
| <span class="sourceLineNo">091</span> private static final String MULTI_MUTATIONS = "multi.mutations";<a name="line.91"></a> |
| <span class="sourceLineNo">092</span> private static final String MULTI_SERVICE_CALLS = "multi.service_calls";<a name="line.92"></a> |
| <span class="sourceLineNo">093</span><a name="line.93"></a> |
| <span class="sourceLineNo">094</span> private final boolean authorize;<a name="line.94"></a> |
| <span class="sourceLineNo">095</span> private volatile boolean isOnlineLogProviderEnabled;<a name="line.95"></a> |
| <span class="sourceLineNo">096</span> protected boolean isSecurityEnabled;<a name="line.96"></a> |
| <span class="sourceLineNo">097</span><a name="line.97"></a> |
| <span class="sourceLineNo">098</span> public static final byte CURRENT_VERSION = 0;<a name="line.98"></a> |
| <span class="sourceLineNo">099</span><a name="line.99"></a> |
| <span class="sourceLineNo">100</span> /**<a name="line.100"></a> |
| <span class="sourceLineNo">101</span> * Whether we allow a fallback to SIMPLE auth for insecure clients when security is enabled.<a name="line.101"></a> |
| <span class="sourceLineNo">102</span> */<a name="line.102"></a> |
| <span class="sourceLineNo">103</span> public static final String FALLBACK_TO_INSECURE_CLIENT_AUTH =<a name="line.103"></a> |
| <span class="sourceLineNo">104</span> "hbase.ipc.server.fallback-to-simple-auth-allowed";<a name="line.104"></a> |
| <span class="sourceLineNo">105</span><a name="line.105"></a> |
| <span class="sourceLineNo">106</span> /**<a name="line.106"></a> |
| <span class="sourceLineNo">107</span> * How many calls/handler are allowed in the queue.<a name="line.107"></a> |
| <span class="sourceLineNo">108</span> */<a name="line.108"></a> |
| <span class="sourceLineNo">109</span> protected static final int DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER = 10;<a name="line.109"></a> |
| <span class="sourceLineNo">110</span><a name="line.110"></a> |
| <span class="sourceLineNo">111</span> protected final CellBlockBuilder cellBlockBuilder;<a name="line.111"></a> |
| <span class="sourceLineNo">112</span><a name="line.112"></a> |
| <span class="sourceLineNo">113</span> protected static final String AUTH_FAILED_FOR = "Auth failed for ";<a name="line.113"></a> |
| <span class="sourceLineNo">114</span> protected static final String AUTH_SUCCESSFUL_FOR = "Auth successful for ";<a name="line.114"></a> |
| <span class="sourceLineNo">115</span> protected static final Logger AUDITLOG =<a name="line.115"></a> |
| <span class="sourceLineNo">116</span> LoggerFactory.getLogger("SecurityLogger." + Server.class.getName());<a name="line.116"></a> |
| <span class="sourceLineNo">117</span> protected SecretManager<TokenIdentifier> secretManager;<a name="line.117"></a> |
| <span class="sourceLineNo">118</span> protected final Map<String, String> saslProps;<a name="line.118"></a> |
| <span class="sourceLineNo">119</span><a name="line.119"></a> |
| <span class="sourceLineNo">120</span> protected ServiceAuthorizationManager authManager;<a name="line.120"></a> |
| <span class="sourceLineNo">121</span><a name="line.121"></a> |
| <span class="sourceLineNo">122</span> /**<a name="line.122"></a> |
| <span class="sourceLineNo">123</span> * This is set to Call object before Handler invokes an RPC and ybdie after the call returns.<a name="line.123"></a> |
| <span class="sourceLineNo">124</span> */<a name="line.124"></a> |
| <span class="sourceLineNo">125</span> protected static final ThreadLocal<RpcCall> CurCall = new ThreadLocal<>();<a name="line.125"></a> |
| <span class="sourceLineNo">126</span><a name="line.126"></a> |
| <span class="sourceLineNo">127</span> /** Keeps MonitoredRPCHandler per handler thread. */<a name="line.127"></a> |
| <span class="sourceLineNo">128</span> protected static final ThreadLocal<MonitoredRPCHandler> MONITORED_RPC = new ThreadLocal<>();<a name="line.128"></a> |
| <span class="sourceLineNo">129</span><a name="line.129"></a> |
| <span class="sourceLineNo">130</span> protected final InetSocketAddress bindAddress;<a name="line.130"></a> |
| <span class="sourceLineNo">131</span><a name="line.131"></a> |
| <span class="sourceLineNo">132</span> protected MetricsHBaseServer metrics;<a name="line.132"></a> |
| <span class="sourceLineNo">133</span><a name="line.133"></a> |
| <span class="sourceLineNo">134</span> protected final Configuration conf;<a name="line.134"></a> |
| <span class="sourceLineNo">135</span><a name="line.135"></a> |
| <span class="sourceLineNo">136</span> /**<a name="line.136"></a> |
| <span class="sourceLineNo">137</span> * Maximum size in bytes of the currently queued and running Calls. If a new Call puts us over<a name="line.137"></a> |
| <span class="sourceLineNo">138</span> * this size, then we will reject the call (after parsing it though). It will go back to the<a name="line.138"></a> |
| <span class="sourceLineNo">139</span> * client and client will retry. Set this size with "hbase.ipc.server.max.callqueue.size". The<a name="line.139"></a> |
| <span class="sourceLineNo">140</span> * call queue size gets incremented after we parse a call and before we add it to the queue of<a name="line.140"></a> |
| <span class="sourceLineNo">141</span> * calls for the scheduler to use. It get decremented after we have 'run' the Call. The current<a name="line.141"></a> |
| <span class="sourceLineNo">142</span> * size is kept in {@link #callQueueSizeInBytes}.<a name="line.142"></a> |
| <span class="sourceLineNo">143</span> * @see #callQueueSizeInBytes<a name="line.143"></a> |
| <span class="sourceLineNo">144</span> * @see #DEFAULT_MAX_CALLQUEUE_SIZE<a name="line.144"></a> |
| <span class="sourceLineNo">145</span> */<a name="line.145"></a> |
| <span class="sourceLineNo">146</span> protected final long maxQueueSizeInBytes;<a name="line.146"></a> |
| <span class="sourceLineNo">147</span> protected static final int DEFAULT_MAX_CALLQUEUE_SIZE = 1024 * 1024 * 1024;<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> * This is a running count of the size in bytes of all outstanding calls whether currently<a name="line.150"></a> |
| <span class="sourceLineNo">151</span> * executing or queued waiting to be run.<a name="line.151"></a> |
| <span class="sourceLineNo">152</span> */<a name="line.152"></a> |
| <span class="sourceLineNo">153</span> protected final LongAdder callQueueSizeInBytes = new LongAdder();<a name="line.153"></a> |
| <span class="sourceLineNo">154</span><a name="line.154"></a> |
| <span class="sourceLineNo">155</span> protected final boolean tcpNoDelay; // if T then disable Nagle's Algorithm<a name="line.155"></a> |
| <span class="sourceLineNo">156</span> protected final boolean tcpKeepAlive; // if T then use keepalives<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> * This flag is used to indicate to sub threads when they should go down. When we call<a name="line.159"></a> |
| <span class="sourceLineNo">160</span> * {@link #start()}, all threads started will consult this flag on whether they should keep going.<a name="line.160"></a> |
| <span class="sourceLineNo">161</span> * It is set to false when {@link #stop()} is called.<a name="line.161"></a> |
| <span class="sourceLineNo">162</span> */<a name="line.162"></a> |
| <span class="sourceLineNo">163</span> volatile boolean running = true;<a name="line.163"></a> |
| <span class="sourceLineNo">164</span><a name="line.164"></a> |
| <span class="sourceLineNo">165</span> /**<a name="line.165"></a> |
| <span class="sourceLineNo">166</span> * This flag is set to true after all threads are up and 'running' and the server is then opened<a name="line.166"></a> |
| <span class="sourceLineNo">167</span> * for business by the call to {@link #start()}.<a name="line.167"></a> |
| <span class="sourceLineNo">168</span> */<a name="line.168"></a> |
| <span class="sourceLineNo">169</span> volatile boolean started = false;<a name="line.169"></a> |
| <span class="sourceLineNo">170</span><a name="line.170"></a> |
| <span class="sourceLineNo">171</span> protected AuthenticationTokenSecretManager authTokenSecretMgr = null;<a name="line.171"></a> |
| <span class="sourceLineNo">172</span><a name="line.172"></a> |
| <span class="sourceLineNo">173</span> protected HBaseRPCErrorHandler errorHandler = null;<a name="line.173"></a> |
| <span class="sourceLineNo">174</span><a name="line.174"></a> |
| <span class="sourceLineNo">175</span> public static final String MAX_REQUEST_SIZE = "hbase.ipc.max.request.size";<a name="line.175"></a> |
| <span class="sourceLineNo">176</span><a name="line.176"></a> |
| <span class="sourceLineNo">177</span> protected static final String WARN_RESPONSE_TIME = "hbase.ipc.warn.response.time";<a name="line.177"></a> |
| <span class="sourceLineNo">178</span> protected static final String WARN_RESPONSE_SIZE = "hbase.ipc.warn.response.size";<a name="line.178"></a> |
| <span class="sourceLineNo">179</span> protected static final String WARN_SCAN_RESPONSE_TIME = "hbase.ipc.warn.response.time.scan";<a name="line.179"></a> |
| <span class="sourceLineNo">180</span> protected static final String WARN_SCAN_RESPONSE_SIZE = "hbase.ipc.warn.response.size.scan";<a name="line.180"></a> |
| <span class="sourceLineNo">181</span><a name="line.181"></a> |
| <span class="sourceLineNo">182</span> /**<a name="line.182"></a> |
| <span class="sourceLineNo">183</span> * Minimum allowable timeout (in milliseconds) in rpc request's header. This configuration exists<a name="line.183"></a> |
| <span class="sourceLineNo">184</span> * to prevent the rpc service regarding this request as timeout immediately.<a name="line.184"></a> |
| <span class="sourceLineNo">185</span> */<a name="line.185"></a> |
| <span class="sourceLineNo">186</span> protected static final String MIN_CLIENT_REQUEST_TIMEOUT = "hbase.ipc.min.client.request.timeout";<a name="line.186"></a> |
| <span class="sourceLineNo">187</span> protected static final int DEFAULT_MIN_CLIENT_REQUEST_TIMEOUT = 20;<a name="line.187"></a> |
| <span class="sourceLineNo">188</span><a name="line.188"></a> |
| <span class="sourceLineNo">189</span> /** Default value for above params */<a name="line.189"></a> |
| <span class="sourceLineNo">190</span> public static final int DEFAULT_MAX_REQUEST_SIZE = DEFAULT_MAX_CALLQUEUE_SIZE / 4; // 256M<a name="line.190"></a> |
| <span class="sourceLineNo">191</span> protected static final int DEFAULT_WARN_RESPONSE_TIME = 10000; // milliseconds<a name="line.191"></a> |
| <span class="sourceLineNo">192</span> protected static final int DEFAULT_WARN_RESPONSE_SIZE = 100 * 1024 * 1024;<a name="line.192"></a> |
| <span class="sourceLineNo">193</span><a name="line.193"></a> |
| <span class="sourceLineNo">194</span> protected static final int DEFAULT_TRACE_LOG_MAX_LENGTH = 1000;<a name="line.194"></a> |
| <span class="sourceLineNo">195</span> protected static final String TRACE_LOG_MAX_LENGTH = "hbase.ipc.trace.log.max.length";<a name="line.195"></a> |
| <span class="sourceLineNo">196</span> protected static final String KEY_WORD_TRUNCATED = " <TRUNCATED>";<a name="line.196"></a> |
| <span class="sourceLineNo">197</span><a name="line.197"></a> |
| <span class="sourceLineNo">198</span> protected static final Gson GSON = GsonUtil.createGsonWithDisableHtmlEscaping().create();<a name="line.198"></a> |
| <span class="sourceLineNo">199</span><a name="line.199"></a> |
| <span class="sourceLineNo">200</span> protected final int maxRequestSize;<a name="line.200"></a> |
| <span class="sourceLineNo">201</span> protected volatile int warnResponseTime;<a name="line.201"></a> |
| <span class="sourceLineNo">202</span> protected volatile int warnResponseSize;<a name="line.202"></a> |
| <span class="sourceLineNo">203</span> protected volatile int warnScanResponseTime;<a name="line.203"></a> |
| <span class="sourceLineNo">204</span> protected volatile int warnScanResponseSize;<a name="line.204"></a> |
| <span class="sourceLineNo">205</span><a name="line.205"></a> |
| <span class="sourceLineNo">206</span> protected final int minClientRequestTimeout;<a name="line.206"></a> |
| <span class="sourceLineNo">207</span><a name="line.207"></a> |
| <span class="sourceLineNo">208</span> protected final Server server;<a name="line.208"></a> |
| <span class="sourceLineNo">209</span> protected final List<BlockingServiceAndInterface> services;<a name="line.209"></a> |
| <span class="sourceLineNo">210</span><a name="line.210"></a> |
| <span class="sourceLineNo">211</span> protected final RpcScheduler scheduler;<a name="line.211"></a> |
| <span class="sourceLineNo">212</span><a name="line.212"></a> |
| <span class="sourceLineNo">213</span> protected UserProvider userProvider;<a name="line.213"></a> |
| <span class="sourceLineNo">214</span><a name="line.214"></a> |
| <span class="sourceLineNo">215</span> protected final ByteBuffAllocator bbAllocator;<a name="line.215"></a> |
| <span class="sourceLineNo">216</span><a name="line.216"></a> |
| <span class="sourceLineNo">217</span> protected volatile boolean allowFallbackToSimpleAuth;<a name="line.217"></a> |
| <span class="sourceLineNo">218</span><a name="line.218"></a> |
| <span class="sourceLineNo">219</span> /**<a name="line.219"></a> |
| <span class="sourceLineNo">220</span> * Used to get details for scan with a scanner_id<br/><a name="line.220"></a> |
| <span class="sourceLineNo">221</span> * TODO try to figure out a better way and remove reference from regionserver package later.<a name="line.221"></a> |
| <span class="sourceLineNo">222</span> */<a name="line.222"></a> |
| <span class="sourceLineNo">223</span> private RSRpcServices rsRpcServices;<a name="line.223"></a> |
| <span class="sourceLineNo">224</span><a name="line.224"></a> |
| <span class="sourceLineNo">225</span> /**<a name="line.225"></a> |
| <span class="sourceLineNo">226</span> * Use to add online slowlog responses<a name="line.226"></a> |
| <span class="sourceLineNo">227</span> */<a name="line.227"></a> |
| <span class="sourceLineNo">228</span> private NamedQueueRecorder namedQueueRecorder;<a name="line.228"></a> |
| <span class="sourceLineNo">229</span><a name="line.229"></a> |
| <span class="sourceLineNo">230</span> @FunctionalInterface<a name="line.230"></a> |
| <span class="sourceLineNo">231</span> protected interface CallCleanup {<a name="line.231"></a> |
| <span class="sourceLineNo">232</span> void run();<a name="line.232"></a> |
| <span class="sourceLineNo">233</span> }<a name="line.233"></a> |
| <span class="sourceLineNo">234</span><a name="line.234"></a> |
| <span class="sourceLineNo">235</span> /**<a name="line.235"></a> |
| <span class="sourceLineNo">236</span> * Datastructure for passing a {@link BlockingService} and its associated class of protobuf<a name="line.236"></a> |
| <span class="sourceLineNo">237</span> * service interface. For example, a server that fielded what is defined in the client protobuf<a name="line.237"></a> |
| <span class="sourceLineNo">238</span> * service would pass in an implementation of the client blocking service and then its<a name="line.238"></a> |
| <span class="sourceLineNo">239</span> * ClientService.BlockingInterface.class. Used checking connection setup.<a name="line.239"></a> |
| <span class="sourceLineNo">240</span> */<a name="line.240"></a> |
| <span class="sourceLineNo">241</span> public static class BlockingServiceAndInterface {<a name="line.241"></a> |
| <span class="sourceLineNo">242</span> private final BlockingService service;<a name="line.242"></a> |
| <span class="sourceLineNo">243</span> private final Class<?> serviceInterface;<a name="line.243"></a> |
| <span class="sourceLineNo">244</span><a name="line.244"></a> |
| <span class="sourceLineNo">245</span> public BlockingServiceAndInterface(final BlockingService service,<a name="line.245"></a> |
| <span class="sourceLineNo">246</span> final Class<?> serviceInterface) {<a name="line.246"></a> |
| <span class="sourceLineNo">247</span> this.service = service;<a name="line.247"></a> |
| <span class="sourceLineNo">248</span> this.serviceInterface = serviceInterface;<a name="line.248"></a> |
| <span class="sourceLineNo">249</span> }<a name="line.249"></a> |
| <span class="sourceLineNo">250</span><a name="line.250"></a> |
| <span class="sourceLineNo">251</span> public Class<?> getServiceInterface() {<a name="line.251"></a> |
| <span class="sourceLineNo">252</span> return this.serviceInterface;<a name="line.252"></a> |
| <span class="sourceLineNo">253</span> }<a name="line.253"></a> |
| <span class="sourceLineNo">254</span><a name="line.254"></a> |
| <span class="sourceLineNo">255</span> public BlockingService getBlockingService() {<a name="line.255"></a> |
| <span class="sourceLineNo">256</span> return this.service;<a name="line.256"></a> |
| <span class="sourceLineNo">257</span> }<a name="line.257"></a> |
| <span class="sourceLineNo">258</span> }<a name="line.258"></a> |
| <span class="sourceLineNo">259</span><a name="line.259"></a> |
| <span class="sourceLineNo">260</span> /**<a name="line.260"></a> |
| <span class="sourceLineNo">261</span> * Constructs a server listening on the named port and address.<a name="line.261"></a> |
| <span class="sourceLineNo">262</span> * @param server hosting instance of {@link Server}. We will do authentications if an<a name="line.262"></a> |
| <span class="sourceLineNo">263</span> * instance else pass null for no authentication check.<a name="line.263"></a> |
| <span class="sourceLineNo">264</span> * @param name Used keying this rpc servers' metrics and for naming the Listener<a name="line.264"></a> |
| <span class="sourceLineNo">265</span> * thread.<a name="line.265"></a> |
| <span class="sourceLineNo">266</span> * @param services A list of services.<a name="line.266"></a> |
| <span class="sourceLineNo">267</span> * @param bindAddress Where to listen<a name="line.267"></a> |
| <span class="sourceLineNo">268</span> * @param reservoirEnabled Enable ByteBufferPool or not.<a name="line.268"></a> |
| <span class="sourceLineNo">269</span> */<a name="line.269"></a> |
| <span class="sourceLineNo">270</span> public RpcServer(final Server server, final String name,<a name="line.270"></a> |
| <span class="sourceLineNo">271</span> final List<BlockingServiceAndInterface> services, final InetSocketAddress bindAddress,<a name="line.271"></a> |
| <span class="sourceLineNo">272</span> Configuration conf, RpcScheduler scheduler, boolean reservoirEnabled) throws IOException {<a name="line.272"></a> |
| <span class="sourceLineNo">273</span> this.bbAllocator = ByteBuffAllocator.create(conf, reservoirEnabled);<a name="line.273"></a> |
| <span class="sourceLineNo">274</span> this.server = server;<a name="line.274"></a> |
| <span class="sourceLineNo">275</span> this.services = services;<a name="line.275"></a> |
| <span class="sourceLineNo">276</span> this.bindAddress = bindAddress;<a name="line.276"></a> |
| <span class="sourceLineNo">277</span> this.conf = conf;<a name="line.277"></a> |
| <span class="sourceLineNo">278</span> // See declaration above for documentation on what this size is.<a name="line.278"></a> |
| <span class="sourceLineNo">279</span> this.maxQueueSizeInBytes =<a name="line.279"></a> |
| <span class="sourceLineNo">280</span> this.conf.getLong("hbase.ipc.server.max.callqueue.size", DEFAULT_MAX_CALLQUEUE_SIZE);<a name="line.280"></a> |
| <span class="sourceLineNo">281</span><a name="line.281"></a> |
| <span class="sourceLineNo">282</span> this.warnResponseTime = getWarnResponseTime(conf);<a name="line.282"></a> |
| <span class="sourceLineNo">283</span> this.warnResponseSize = getWarnResponseSize(conf);<a name="line.283"></a> |
| <span class="sourceLineNo">284</span> this.warnScanResponseTime = getWarnScanResponseTime(conf);<a name="line.284"></a> |
| <span class="sourceLineNo">285</span> this.warnScanResponseSize = getWarnScanResponseSize(conf);<a name="line.285"></a> |
| <span class="sourceLineNo">286</span> this.minClientRequestTimeout =<a name="line.286"></a> |
| <span class="sourceLineNo">287</span> conf.getInt(MIN_CLIENT_REQUEST_TIMEOUT, DEFAULT_MIN_CLIENT_REQUEST_TIMEOUT);<a name="line.287"></a> |
| <span class="sourceLineNo">288</span> this.maxRequestSize = conf.getInt(MAX_REQUEST_SIZE, DEFAULT_MAX_REQUEST_SIZE);<a name="line.288"></a> |
| <span class="sourceLineNo">289</span><a name="line.289"></a> |
| <span class="sourceLineNo">290</span> this.metrics = new MetricsHBaseServer(name, new MetricsHBaseServerWrapperImpl(this));<a name="line.290"></a> |
| <span class="sourceLineNo">291</span> this.tcpNoDelay = conf.getBoolean("hbase.ipc.server.tcpnodelay", true);<a name="line.291"></a> |
| <span class="sourceLineNo">292</span> this.tcpKeepAlive = conf.getBoolean("hbase.ipc.server.tcpkeepalive", true);<a name="line.292"></a> |
| <span class="sourceLineNo">293</span><a name="line.293"></a> |
| <span class="sourceLineNo">294</span> this.cellBlockBuilder = new CellBlockBuilder(conf);<a name="line.294"></a> |
| <span class="sourceLineNo">295</span><a name="line.295"></a> |
| <span class="sourceLineNo">296</span> this.authorize = conf.getBoolean(HADOOP_SECURITY_AUTHORIZATION, false);<a name="line.296"></a> |
| <span class="sourceLineNo">297</span> this.userProvider = UserProvider.instantiate(conf);<a name="line.297"></a> |
| <span class="sourceLineNo">298</span> this.isSecurityEnabled = userProvider.isHBaseSecurityEnabled();<a name="line.298"></a> |
| <span class="sourceLineNo">299</span> if (isSecurityEnabled) {<a name="line.299"></a> |
| <span class="sourceLineNo">300</span> saslProps = SaslUtil.initSaslProperties(conf.get("hbase.rpc.protection",<a name="line.300"></a> |
| <span class="sourceLineNo">301</span> QualityOfProtection.AUTHENTICATION.name().toLowerCase(Locale.ROOT)));<a name="line.301"></a> |
| <span class="sourceLineNo">302</span> } else {<a name="line.302"></a> |
| <span class="sourceLineNo">303</span> saslProps = Collections.emptyMap();<a name="line.303"></a> |
| <span class="sourceLineNo">304</span> }<a name="line.304"></a> |
| <span class="sourceLineNo">305</span><a name="line.305"></a> |
| <span class="sourceLineNo">306</span> this.isOnlineLogProviderEnabled = getIsOnlineLogProviderEnabled(conf);<a name="line.306"></a> |
| <span class="sourceLineNo">307</span> this.scheduler = scheduler;<a name="line.307"></a> |
| <span class="sourceLineNo">308</span> }<a name="line.308"></a> |
| <span class="sourceLineNo">309</span><a name="line.309"></a> |
| <span class="sourceLineNo">310</span> @Override<a name="line.310"></a> |
| <span class="sourceLineNo">311</span> public void onConfigurationChange(Configuration newConf) {<a name="line.311"></a> |
| <span class="sourceLineNo">312</span> initReconfigurable(newConf);<a name="line.312"></a> |
| <span class="sourceLineNo">313</span> if (scheduler instanceof ConfigurationObserver) {<a name="line.313"></a> |
| <span class="sourceLineNo">314</span> ((ConfigurationObserver) scheduler).onConfigurationChange(newConf);<a name="line.314"></a> |
| <span class="sourceLineNo">315</span> }<a name="line.315"></a> |
| <span class="sourceLineNo">316</span> if (authorize) {<a name="line.316"></a> |
| <span class="sourceLineNo">317</span> refreshAuthManager(newConf, new HBasePolicyProvider());<a name="line.317"></a> |
| <span class="sourceLineNo">318</span> }<a name="line.318"></a> |
| <span class="sourceLineNo">319</span> refreshSlowLogConfiguration(newConf);<a name="line.319"></a> |
| <span class="sourceLineNo">320</span> }<a name="line.320"></a> |
| <span class="sourceLineNo">321</span><a name="line.321"></a> |
| <span class="sourceLineNo">322</span> private void refreshSlowLogConfiguration(Configuration newConf) {<a name="line.322"></a> |
| <span class="sourceLineNo">323</span> boolean newIsOnlineLogProviderEnabled = getIsOnlineLogProviderEnabled(newConf);<a name="line.323"></a> |
| <span class="sourceLineNo">324</span> if (isOnlineLogProviderEnabled != newIsOnlineLogProviderEnabled) {<a name="line.324"></a> |
| <span class="sourceLineNo">325</span> isOnlineLogProviderEnabled = newIsOnlineLogProviderEnabled;<a name="line.325"></a> |
| <span class="sourceLineNo">326</span> }<a name="line.326"></a> |
| <span class="sourceLineNo">327</span> int newWarnResponseTime = getWarnResponseTime(newConf);<a name="line.327"></a> |
| <span class="sourceLineNo">328</span> if (warnResponseTime != newWarnResponseTime) {<a name="line.328"></a> |
| <span class="sourceLineNo">329</span> warnResponseTime = newWarnResponseTime;<a name="line.329"></a> |
| <span class="sourceLineNo">330</span> }<a name="line.330"></a> |
| <span class="sourceLineNo">331</span> int newWarnResponseSize = getWarnResponseSize(newConf);<a name="line.331"></a> |
| <span class="sourceLineNo">332</span> if (warnResponseSize != newWarnResponseSize) {<a name="line.332"></a> |
| <span class="sourceLineNo">333</span> warnResponseSize = newWarnResponseSize;<a name="line.333"></a> |
| <span class="sourceLineNo">334</span> }<a name="line.334"></a> |
| <span class="sourceLineNo">335</span> int newWarnResponseTimeScan = getWarnScanResponseTime(newConf);<a name="line.335"></a> |
| <span class="sourceLineNo">336</span> if (warnScanResponseTime != newWarnResponseTimeScan) {<a name="line.336"></a> |
| <span class="sourceLineNo">337</span> warnScanResponseTime = newWarnResponseTimeScan;<a name="line.337"></a> |
| <span class="sourceLineNo">338</span> }<a name="line.338"></a> |
| <span class="sourceLineNo">339</span> int newWarnScanResponseSize = getWarnScanResponseSize(newConf);<a name="line.339"></a> |
| <span class="sourceLineNo">340</span> if (warnScanResponseSize != newWarnScanResponseSize) {<a name="line.340"></a> |
| <span class="sourceLineNo">341</span> warnScanResponseSize = newWarnScanResponseSize;<a name="line.341"></a> |
| <span class="sourceLineNo">342</span> }<a name="line.342"></a> |
| <span class="sourceLineNo">343</span> }<a name="line.343"></a> |
| <span class="sourceLineNo">344</span><a name="line.344"></a> |
| <span class="sourceLineNo">345</span> private static boolean getIsOnlineLogProviderEnabled(Configuration conf) {<a name="line.345"></a> |
| <span class="sourceLineNo">346</span> return conf.getBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY,<a name="line.346"></a> |
| <span class="sourceLineNo">347</span> HConstants.DEFAULT_ONLINE_LOG_PROVIDER_ENABLED);<a name="line.347"></a> |
| <span class="sourceLineNo">348</span> }<a name="line.348"></a> |
| <span class="sourceLineNo">349</span><a name="line.349"></a> |
| <span class="sourceLineNo">350</span> private static int getWarnResponseTime(Configuration conf) {<a name="line.350"></a> |
| <span class="sourceLineNo">351</span> return conf.getInt(WARN_RESPONSE_TIME, DEFAULT_WARN_RESPONSE_TIME);<a name="line.351"></a> |
| <span class="sourceLineNo">352</span> }<a name="line.352"></a> |
| <span class="sourceLineNo">353</span><a name="line.353"></a> |
| <span class="sourceLineNo">354</span> private static int getWarnResponseSize(Configuration conf) {<a name="line.354"></a> |
| <span class="sourceLineNo">355</span> return conf.getInt(WARN_RESPONSE_SIZE, DEFAULT_WARN_RESPONSE_SIZE);<a name="line.355"></a> |
| <span class="sourceLineNo">356</span> }<a name="line.356"></a> |
| <span class="sourceLineNo">357</span><a name="line.357"></a> |
| <span class="sourceLineNo">358</span> private static int getWarnScanResponseTime(Configuration conf) {<a name="line.358"></a> |
| <span class="sourceLineNo">359</span> return conf.getInt(WARN_SCAN_RESPONSE_TIME, getWarnResponseTime(conf));<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> private static int getWarnScanResponseSize(Configuration conf) {<a name="line.362"></a> |
| <span class="sourceLineNo">363</span> return conf.getInt(WARN_SCAN_RESPONSE_SIZE, getWarnResponseSize(conf));<a name="line.363"></a> |
| <span class="sourceLineNo">364</span> }<a name="line.364"></a> |
| <span class="sourceLineNo">365</span><a name="line.365"></a> |
| <span class="sourceLineNo">366</span> protected void initReconfigurable(Configuration confToLoad) {<a name="line.366"></a> |
| <span class="sourceLineNo">367</span> this.allowFallbackToSimpleAuth = confToLoad.getBoolean(FALLBACK_TO_INSECURE_CLIENT_AUTH, false);<a name="line.367"></a> |
| <span class="sourceLineNo">368</span> if (isSecurityEnabled && allowFallbackToSimpleAuth) {<a name="line.368"></a> |
| <span class="sourceLineNo">369</span> LOG.warn("********* WARNING! *********");<a name="line.369"></a> |
| <span class="sourceLineNo">370</span> LOG.warn("This server is configured to allow connections from INSECURE clients");<a name="line.370"></a> |
| <span class="sourceLineNo">371</span> LOG.warn("(" + FALLBACK_TO_INSECURE_CLIENT_AUTH + " = true).");<a name="line.371"></a> |
| <span class="sourceLineNo">372</span> LOG.warn("While this option is enabled, client identities cannot be secured, and user");<a name="line.372"></a> |
| <span class="sourceLineNo">373</span> LOG.warn("impersonation is possible!");<a name="line.373"></a> |
| <span class="sourceLineNo">374</span> LOG.warn("For secure operation, please disable SIMPLE authentication as soon as possible,");<a name="line.374"></a> |
| <span class="sourceLineNo">375</span> LOG.warn("by setting " + FALLBACK_TO_INSECURE_CLIENT_AUTH + " = false in hbase-site.xml");<a name="line.375"></a> |
| <span class="sourceLineNo">376</span> LOG.warn("****************************");<a name="line.376"></a> |
| <span class="sourceLineNo">377</span> }<a name="line.377"></a> |
| <span class="sourceLineNo">378</span> }<a name="line.378"></a> |
| <span class="sourceLineNo">379</span><a name="line.379"></a> |
| <span class="sourceLineNo">380</span> Configuration getConf() {<a name="line.380"></a> |
| <span class="sourceLineNo">381</span> return conf;<a name="line.381"></a> |
| <span class="sourceLineNo">382</span> }<a name="line.382"></a> |
| <span class="sourceLineNo">383</span><a name="line.383"></a> |
| <span class="sourceLineNo">384</span> @Override<a name="line.384"></a> |
| <span class="sourceLineNo">385</span> public boolean isStarted() {<a name="line.385"></a> |
| <span class="sourceLineNo">386</span> return this.started;<a name="line.386"></a> |
| <span class="sourceLineNo">387</span> }<a name="line.387"></a> |
| <span class="sourceLineNo">388</span><a name="line.388"></a> |
| <span class="sourceLineNo">389</span> @Override<a name="line.389"></a> |
| <span class="sourceLineNo">390</span> public synchronized void refreshAuthManager(Configuration conf, PolicyProvider pp) {<a name="line.390"></a> |
| <span class="sourceLineNo">391</span> // Ignore warnings that this should be accessed in a static way instead of via an instance;<a name="line.391"></a> |
| <span class="sourceLineNo">392</span> // it'll break if you go via static route.<a name="line.392"></a> |
| <span class="sourceLineNo">393</span> System.setProperty("hadoop.policy.file", "hbase-policy.xml");<a name="line.393"></a> |
| <span class="sourceLineNo">394</span> this.authManager.refresh(conf, pp);<a name="line.394"></a> |
| <span class="sourceLineNo">395</span> LOG.info("Refreshed hbase-policy.xml successfully");<a name="line.395"></a> |
| <span class="sourceLineNo">396</span> ProxyUsers.refreshSuperUserGroupsConfiguration(conf);<a name="line.396"></a> |
| <span class="sourceLineNo">397</span> LOG.info("Refreshed super and proxy users successfully");<a name="line.397"></a> |
| <span class="sourceLineNo">398</span> }<a name="line.398"></a> |
| <span class="sourceLineNo">399</span><a name="line.399"></a> |
| <span class="sourceLineNo">400</span> protected AuthenticationTokenSecretManager createSecretManager() {<a name="line.400"></a> |
| <span class="sourceLineNo">401</span> if (!isSecurityEnabled) return null;<a name="line.401"></a> |
| <span class="sourceLineNo">402</span> if (server == null) return null;<a name="line.402"></a> |
| <span class="sourceLineNo">403</span> Configuration conf = server.getConfiguration();<a name="line.403"></a> |
| <span class="sourceLineNo">404</span> long keyUpdateInterval = conf.getLong("hbase.auth.key.update.interval", 24 * 60 * 60 * 1000);<a name="line.404"></a> |
| <span class="sourceLineNo">405</span> long maxAge = conf.getLong("hbase.auth.token.max.lifetime", 7 * 24 * 60 * 60 * 1000);<a name="line.405"></a> |
| <span class="sourceLineNo">406</span> return new AuthenticationTokenSecretManager(conf, server.getZooKeeper(),<a name="line.406"></a> |
| <span class="sourceLineNo">407</span> server.getServerName().toString(), keyUpdateInterval, maxAge);<a name="line.407"></a> |
| <span class="sourceLineNo">408</span> }<a name="line.408"></a> |
| <span class="sourceLineNo">409</span><a name="line.409"></a> |
| <span class="sourceLineNo">410</span> public SecretManager<? extends TokenIdentifier> getSecretManager() {<a name="line.410"></a> |
| <span class="sourceLineNo">411</span> return this.secretManager;<a name="line.411"></a> |
| <span class="sourceLineNo">412</span> }<a name="line.412"></a> |
| <span class="sourceLineNo">413</span><a name="line.413"></a> |
| <span class="sourceLineNo">414</span> @SuppressWarnings("unchecked")<a name="line.414"></a> |
| <span class="sourceLineNo">415</span> public void setSecretManager(SecretManager<? extends TokenIdentifier> secretManager) {<a name="line.415"></a> |
| <span class="sourceLineNo">416</span> this.secretManager = (SecretManager<TokenIdentifier>) secretManager;<a name="line.416"></a> |
| <span class="sourceLineNo">417</span> }<a name="line.417"></a> |
| <span class="sourceLineNo">418</span><a name="line.418"></a> |
| <span class="sourceLineNo">419</span> /**<a name="line.419"></a> |
| <span class="sourceLineNo">420</span> * This is a server side method, which is invoked over RPC. On success the return response has<a name="line.420"></a> |
| <span class="sourceLineNo">421</span> * protobuf response payload. On failure, the exception name and the stack trace are returned in<a name="line.421"></a> |
| <span class="sourceLineNo">422</span> * the protobuf response.<a name="line.422"></a> |
| <span class="sourceLineNo">423</span> */<a name="line.423"></a> |
| <span class="sourceLineNo">424</span> @Override<a name="line.424"></a> |
| <span class="sourceLineNo">425</span> public Pair<Message, CellScanner> call(RpcCall call, MonitoredRPCHandler status)<a name="line.425"></a> |
| <span class="sourceLineNo">426</span> throws IOException {<a name="line.426"></a> |
| <span class="sourceLineNo">427</span> try {<a name="line.427"></a> |
| <span class="sourceLineNo">428</span> MethodDescriptor md = call.getMethod();<a name="line.428"></a> |
| <span class="sourceLineNo">429</span> Message param = call.getParam();<a name="line.429"></a> |
| <span class="sourceLineNo">430</span> status.setRPC(md.getName(), new Object[] { param }, call.getReceiveTime());<a name="line.430"></a> |
| <span class="sourceLineNo">431</span> // TODO: Review after we add in encoded data blocks.<a name="line.431"></a> |
| <span class="sourceLineNo">432</span> status.setRPCPacket(param);<a name="line.432"></a> |
| <span class="sourceLineNo">433</span> status.resume("Servicing call");<a name="line.433"></a> |
| <span class="sourceLineNo">434</span> // get an instance of the method arg type<a name="line.434"></a> |
| <span class="sourceLineNo">435</span> HBaseRpcController controller = new HBaseRpcControllerImpl(call.getCellScanner());<a name="line.435"></a> |
| <span class="sourceLineNo">436</span> controller.setCallTimeout(call.getTimeout());<a name="line.436"></a> |
| <span class="sourceLineNo">437</span> Message result = call.getService().callBlockingMethod(md, controller, param);<a name="line.437"></a> |
| <span class="sourceLineNo">438</span> long receiveTime = call.getReceiveTime();<a name="line.438"></a> |
| <span class="sourceLineNo">439</span> long startTime = call.getStartTime();<a name="line.439"></a> |
| <span class="sourceLineNo">440</span> long endTime = EnvironmentEdgeManager.currentTime();<a name="line.440"></a> |
| <span class="sourceLineNo">441</span> int processingTime = (int) (endTime - startTime);<a name="line.441"></a> |
| <span class="sourceLineNo">442</span> int qTime = (int) (startTime - receiveTime);<a name="line.442"></a> |
| <span class="sourceLineNo">443</span> int totalTime = (int) (endTime - receiveTime);<a name="line.443"></a> |
| <span class="sourceLineNo">444</span> if (LOG.isTraceEnabled()) {<a name="line.444"></a> |
| <span class="sourceLineNo">445</span> LOG.trace(<a name="line.445"></a> |
| <span class="sourceLineNo">446</span> "{}, response: {}, receiveTime: {}, queueTime: {}, processingTime: {}, totalTime: {}",<a name="line.446"></a> |
| <span class="sourceLineNo">447</span> CurCall.get().toString(), TextFormat.shortDebugString(result),<a name="line.447"></a> |
| <span class="sourceLineNo">448</span> CurCall.get().getReceiveTime(), qTime, processingTime, totalTime);<a name="line.448"></a> |
| <span class="sourceLineNo">449</span> }<a name="line.449"></a> |
| <span class="sourceLineNo">450</span> // Use the raw request call size for now.<a name="line.450"></a> |
| <span class="sourceLineNo">451</span> long requestSize = call.getSize();<a name="line.451"></a> |
| <span class="sourceLineNo">452</span> long responseSize = result.getSerializedSize();<a name="line.452"></a> |
| <span class="sourceLineNo">453</span> long responseBlockSize = call.getBlockBytesScanned();<a name="line.453"></a> |
| <span class="sourceLineNo">454</span> if (call.isClientCellBlockSupported()) {<a name="line.454"></a> |
| <span class="sourceLineNo">455</span> // Include the payload size in HBaseRpcController<a name="line.455"></a> |
| <span class="sourceLineNo">456</span> responseSize += call.getResponseCellSize();<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> metrics.dequeuedCall(qTime);<a name="line.459"></a> |
| <span class="sourceLineNo">460</span> metrics.processedCall(processingTime);<a name="line.460"></a> |
| <span class="sourceLineNo">461</span> metrics.totalCall(totalTime);<a name="line.461"></a> |
| <span class="sourceLineNo">462</span> metrics.receivedRequest(requestSize);<a name="line.462"></a> |
| <span class="sourceLineNo">463</span> metrics.sentResponse(responseSize);<a name="line.463"></a> |
| <span class="sourceLineNo">464</span> // log any RPC responses that are slower than the configured warn<a name="line.464"></a> |
| <span class="sourceLineNo">465</span> // response time or larger than configured warning size<a name="line.465"></a> |
| <span class="sourceLineNo">466</span> boolean tooSlow = isTooSlow(call, processingTime);<a name="line.466"></a> |
| <span class="sourceLineNo">467</span> boolean tooLarge = isTooLarge(call, responseSize, responseBlockSize);<a name="line.467"></a> |
| <span class="sourceLineNo">468</span> if (tooSlow || tooLarge) {<a name="line.468"></a> |
| <span class="sourceLineNo">469</span> final String userName = call.getRequestUserName().orElse(StringUtils.EMPTY);<a name="line.469"></a> |
| <span class="sourceLineNo">470</span> // when tagging, we let TooLarge trump TooSmall to keep output simple<a name="line.470"></a> |
| <span class="sourceLineNo">471</span> // note that large responses will often also be slow.<a name="line.471"></a> |
| <span class="sourceLineNo">472</span> logResponse(param, md.getName(), md.getName() + "(" + param.getClass().getName() + ")",<a name="line.472"></a> |
| <span class="sourceLineNo">473</span> tooLarge, tooSlow, status.getClient(), startTime, processingTime, qTime, responseSize,<a name="line.473"></a> |
| <span class="sourceLineNo">474</span> responseBlockSize, userName);<a name="line.474"></a> |
| <span class="sourceLineNo">475</span> if (this.namedQueueRecorder != null && this.isOnlineLogProviderEnabled) {<a name="line.475"></a> |
| <span class="sourceLineNo">476</span> // send logs to ring buffer owned by slowLogRecorder<a name="line.476"></a> |
| <span class="sourceLineNo">477</span> final String className =<a name="line.477"></a> |
| <span class="sourceLineNo">478</span> server == null ? StringUtils.EMPTY : server.getClass().getSimpleName();<a name="line.478"></a> |
| <span class="sourceLineNo">479</span> this.namedQueueRecorder.addRecord(new RpcLogDetails(call, param, status.getClient(),<a name="line.479"></a> |
| <span class="sourceLineNo">480</span> responseSize, responseBlockSize, className, tooSlow, tooLarge));<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> return new Pair<>(result, controller.cellScanner());<a name="line.483"></a> |
| <span class="sourceLineNo">484</span> } catch (Throwable e) {<a name="line.484"></a> |
| <span class="sourceLineNo">485</span> // The above callBlockingMethod will always return a SE. Strip the SE wrapper before<a name="line.485"></a> |
| <span class="sourceLineNo">486</span> // putting it on the wire. Its needed to adhere to the pb Service Interface but we don't<a name="line.486"></a> |
| <span class="sourceLineNo">487</span> // need to pass it over the wire.<a name="line.487"></a> |
| <span class="sourceLineNo">488</span> if (e instanceof ServiceException) {<a name="line.488"></a> |
| <span class="sourceLineNo">489</span> if (e.getCause() == null) {<a name="line.489"></a> |
| <span class="sourceLineNo">490</span> LOG.debug("Caught a ServiceException with null cause", e);<a name="line.490"></a> |
| <span class="sourceLineNo">491</span> } else {<a name="line.491"></a> |
| <span class="sourceLineNo">492</span> e = e.getCause();<a name="line.492"></a> |
| <span class="sourceLineNo">493</span> }<a name="line.493"></a> |
| <span class="sourceLineNo">494</span> }<a name="line.494"></a> |
| <span class="sourceLineNo">495</span><a name="line.495"></a> |
| <span class="sourceLineNo">496</span> // increment the number of requests that were exceptions.<a name="line.496"></a> |
| <span class="sourceLineNo">497</span> metrics.exception(e);<a name="line.497"></a> |
| <span class="sourceLineNo">498</span><a name="line.498"></a> |
| <span class="sourceLineNo">499</span> if (e instanceof LinkageError) throw new DoNotRetryIOException(e);<a name="line.499"></a> |
| <span class="sourceLineNo">500</span> if (e instanceof IOException) throw (IOException) e;<a name="line.500"></a> |
| <span class="sourceLineNo">501</span> LOG.error("Unexpected throwable object ", e);<a name="line.501"></a> |
| <span class="sourceLineNo">502</span> throw new IOException(e.getMessage(), e);<a name="line.502"></a> |
| <span class="sourceLineNo">503</span> }<a name="line.503"></a> |
| <span class="sourceLineNo">504</span> }<a name="line.504"></a> |
| <span class="sourceLineNo">505</span><a name="line.505"></a> |
| <span class="sourceLineNo">506</span> /**<a name="line.506"></a> |
| <span class="sourceLineNo">507</span> * Logs an RPC response to the LOG file, producing valid JSON objects for client Operations.<a name="line.507"></a> |
| <span class="sourceLineNo">508</span> * @param param The parameters received in the call.<a name="line.508"></a> |
| <span class="sourceLineNo">509</span> * @param methodName The name of the method invoked<a name="line.509"></a> |
| <span class="sourceLineNo">510</span> * @param call The string representation of the call<a name="line.510"></a> |
| <span class="sourceLineNo">511</span> * @param tooLarge To indicate if the event is tooLarge<a name="line.511"></a> |
| <span class="sourceLineNo">512</span> * @param tooSlow To indicate if the event is tooSlow<a name="line.512"></a> |
| <span class="sourceLineNo">513</span> * @param clientAddress The address of the client who made this call.<a name="line.513"></a> |
| <span class="sourceLineNo">514</span> * @param startTime The time that the call was initiated, in ms.<a name="line.514"></a> |
| <span class="sourceLineNo">515</span> * @param processingTime The duration that the call took to run, in ms.<a name="line.515"></a> |
| <span class="sourceLineNo">516</span> * @param qTime The duration that the call spent on the queue prior to being<a name="line.516"></a> |
| <span class="sourceLineNo">517</span> * initiated, in ms.<a name="line.517"></a> |
| <span class="sourceLineNo">518</span> * @param responseSize The size in bytes of the response buffer.<a name="line.518"></a> |
| <span class="sourceLineNo">519</span> * @param blockBytesScanned The size of block bytes scanned to retrieve the response.<a name="line.519"></a> |
| <span class="sourceLineNo">520</span> * @param userName UserName of the current RPC Call<a name="line.520"></a> |
| <span class="sourceLineNo">521</span> */<a name="line.521"></a> |
| <span class="sourceLineNo">522</span> void logResponse(Message param, String methodName, String call, boolean tooLarge, boolean tooSlow,<a name="line.522"></a> |
| <span class="sourceLineNo">523</span> String clientAddress, long startTime, int processingTime, int qTime, long responseSize,<a name="line.523"></a> |
| <span class="sourceLineNo">524</span> long blockBytesScanned, String userName) {<a name="line.524"></a> |
| <span class="sourceLineNo">525</span> final String className = server == null ? StringUtils.EMPTY : server.getClass().getSimpleName();<a name="line.525"></a> |
| <span class="sourceLineNo">526</span> // base information that is reported regardless of type of call<a name="line.526"></a> |
| <span class="sourceLineNo">527</span> Map<String, Object> responseInfo = new HashMap<>();<a name="line.527"></a> |
| <span class="sourceLineNo">528</span> responseInfo.put("starttimems", startTime);<a name="line.528"></a> |
| <span class="sourceLineNo">529</span> responseInfo.put("processingtimems", processingTime);<a name="line.529"></a> |
| <span class="sourceLineNo">530</span> responseInfo.put("queuetimems", qTime);<a name="line.530"></a> |
| <span class="sourceLineNo">531</span> responseInfo.put("responsesize", responseSize);<a name="line.531"></a> |
| <span class="sourceLineNo">532</span> responseInfo.put("blockbytesscanned", blockBytesScanned);<a name="line.532"></a> |
| <span class="sourceLineNo">533</span> responseInfo.put("client", clientAddress);<a name="line.533"></a> |
| <span class="sourceLineNo">534</span> responseInfo.put("class", className);<a name="line.534"></a> |
| <span class="sourceLineNo">535</span> responseInfo.put("method", methodName);<a name="line.535"></a> |
| <span class="sourceLineNo">536</span> responseInfo.put("call", call);<a name="line.536"></a> |
| <span class="sourceLineNo">537</span> // The params could be really big, make sure they don't kill us at WARN<a name="line.537"></a> |
| <span class="sourceLineNo">538</span> String stringifiedParam = ProtobufUtil.getShortTextFormat(param);<a name="line.538"></a> |
| <span class="sourceLineNo">539</span> if (stringifiedParam.length() > 150) {<a name="line.539"></a> |
| <span class="sourceLineNo">540</span> // Truncate to 1000 chars if TRACE is on, else to 150 chars<a name="line.540"></a> |
| <span class="sourceLineNo">541</span> stringifiedParam = truncateTraceLog(stringifiedParam);<a name="line.541"></a> |
| <span class="sourceLineNo">542</span> }<a name="line.542"></a> |
| <span class="sourceLineNo">543</span> responseInfo.put("param", stringifiedParam);<a name="line.543"></a> |
| <span class="sourceLineNo">544</span> if (param instanceof ClientProtos.ScanRequest && rsRpcServices != null) {<a name="line.544"></a> |
| <span class="sourceLineNo">545</span> ClientProtos.ScanRequest request = ((ClientProtos.ScanRequest) param);<a name="line.545"></a> |
| <span class="sourceLineNo">546</span> String scanDetails;<a name="line.546"></a> |
| <span class="sourceLineNo">547</span> if (request.hasScannerId()) {<a name="line.547"></a> |
| <span class="sourceLineNo">548</span> long scannerId = request.getScannerId();<a name="line.548"></a> |
| <span class="sourceLineNo">549</span> scanDetails = rsRpcServices.getScanDetailsWithId(scannerId);<a name="line.549"></a> |
| <span class="sourceLineNo">550</span> } else {<a name="line.550"></a> |
| <span class="sourceLineNo">551</span> scanDetails = rsRpcServices.getScanDetailsWithRequest(request);<a name="line.551"></a> |
| <span class="sourceLineNo">552</span> }<a name="line.552"></a> |
| <span class="sourceLineNo">553</span> if (scanDetails != null) {<a name="line.553"></a> |
| <span class="sourceLineNo">554</span> responseInfo.put("scandetails", scanDetails);<a name="line.554"></a> |
| <span class="sourceLineNo">555</span> }<a name="line.555"></a> |
| <span class="sourceLineNo">556</span> }<a name="line.556"></a> |
| <span class="sourceLineNo">557</span> if (param instanceof ClientProtos.MultiRequest) {<a name="line.557"></a> |
| <span class="sourceLineNo">558</span> int numGets = 0;<a name="line.558"></a> |
| <span class="sourceLineNo">559</span> int numMutations = 0;<a name="line.559"></a> |
| <span class="sourceLineNo">560</span> int numServiceCalls = 0;<a name="line.560"></a> |
| <span class="sourceLineNo">561</span> ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;<a name="line.561"></a> |
| <span class="sourceLineNo">562</span> for (ClientProtos.RegionAction regionAction : multi.getRegionActionList()) {<a name="line.562"></a> |
| <span class="sourceLineNo">563</span> for (ClientProtos.Action action : regionAction.getActionList()) {<a name="line.563"></a> |
| <span class="sourceLineNo">564</span> if (action.hasMutation()) {<a name="line.564"></a> |
| <span class="sourceLineNo">565</span> numMutations++;<a name="line.565"></a> |
| <span class="sourceLineNo">566</span> }<a name="line.566"></a> |
| <span class="sourceLineNo">567</span> if (action.hasGet()) {<a name="line.567"></a> |
| <span class="sourceLineNo">568</span> numGets++;<a name="line.568"></a> |
| <span class="sourceLineNo">569</span> }<a name="line.569"></a> |
| <span class="sourceLineNo">570</span> if (action.hasServiceCall()) {<a name="line.570"></a> |
| <span class="sourceLineNo">571</span> numServiceCalls++;<a name="line.571"></a> |
| <span class="sourceLineNo">572</span> }<a name="line.572"></a> |
| <span class="sourceLineNo">573</span> }<a name="line.573"></a> |
| <span class="sourceLineNo">574</span> }<a name="line.574"></a> |
| <span class="sourceLineNo">575</span> responseInfo.put(MULTI_GETS, numGets);<a name="line.575"></a> |
| <span class="sourceLineNo">576</span> responseInfo.put(MULTI_MUTATIONS, numMutations);<a name="line.576"></a> |
| <span class="sourceLineNo">577</span> responseInfo.put(MULTI_SERVICE_CALLS, numServiceCalls);<a name="line.577"></a> |
| <span class="sourceLineNo">578</span> }<a name="line.578"></a> |
| <span class="sourceLineNo">579</span> final String tag =<a name="line.579"></a> |
| <span class="sourceLineNo">580</span> (tooLarge && tooSlow) ? "TooLarge & TooSlow" : (tooSlow ? "TooSlow" : "TooLarge");<a name="line.580"></a> |
| <span class="sourceLineNo">581</span> LOG.warn("(response" + tag + "): " + GSON.toJson(responseInfo));<a name="line.581"></a> |
| <span class="sourceLineNo">582</span> }<a name="line.582"></a> |
| <span class="sourceLineNo">583</span><a name="line.583"></a> |
| <span class="sourceLineNo">584</span> private boolean isTooSlow(RpcCall call, int processingTime) {<a name="line.584"></a> |
| <span class="sourceLineNo">585</span> long warnResponseTime = call.getParam() instanceof ClientProtos.ScanRequest<a name="line.585"></a> |
| <span class="sourceLineNo">586</span> ? warnScanResponseTime<a name="line.586"></a> |
| <span class="sourceLineNo">587</span> : this.warnResponseTime;<a name="line.587"></a> |
| <span class="sourceLineNo">588</span> return (processingTime > warnResponseTime && warnResponseTime > -1);<a name="line.588"></a> |
| <span class="sourceLineNo">589</span> }<a name="line.589"></a> |
| <span class="sourceLineNo">590</span><a name="line.590"></a> |
| <span class="sourceLineNo">591</span> private boolean isTooLarge(RpcCall call, long responseSize, long responseBlockSize) {<a name="line.591"></a> |
| <span class="sourceLineNo">592</span> long warnResponseSize = call.getParam() instanceof ClientProtos.ScanRequest<a name="line.592"></a> |
| <span class="sourceLineNo">593</span> ? warnScanResponseSize<a name="line.593"></a> |
| <span class="sourceLineNo">594</span> : this.warnResponseSize;<a name="line.594"></a> |
| <span class="sourceLineNo">595</span> return (warnResponseSize > -1<a name="line.595"></a> |
| <span class="sourceLineNo">596</span> && (responseSize > warnResponseSize || responseBlockSize > warnResponseSize));<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> /**<a name="line.599"></a> |
| <span class="sourceLineNo">600</span> * Truncate to number of chars decided by conf hbase.ipc.trace.log.max.length if TRACE is on else<a name="line.600"></a> |
| <span class="sourceLineNo">601</span> * to 150 chars Refer to Jira HBASE-20826 and HBASE-20942<a name="line.601"></a> |
| <span class="sourceLineNo">602</span> * @param strParam stringifiedParam to be truncated<a name="line.602"></a> |
| <span class="sourceLineNo">603</span> * @return truncated trace log string<a name="line.603"></a> |
| <span class="sourceLineNo">604</span> */<a name="line.604"></a> |
| <span class="sourceLineNo">605</span> String truncateTraceLog(String strParam) {<a name="line.605"></a> |
| <span class="sourceLineNo">606</span> if (LOG.isTraceEnabled()) {<a name="line.606"></a> |
| <span class="sourceLineNo">607</span> int traceLogMaxLength = getConf().getInt(TRACE_LOG_MAX_LENGTH, DEFAULT_TRACE_LOG_MAX_LENGTH);<a name="line.607"></a> |
| <span class="sourceLineNo">608</span> int truncatedLength =<a name="line.608"></a> |
| <span class="sourceLineNo">609</span> strParam.length() < traceLogMaxLength ? strParam.length() : traceLogMaxLength;<a name="line.609"></a> |
| <span class="sourceLineNo">610</span> String truncatedFlag = truncatedLength == strParam.length() ? "" : KEY_WORD_TRUNCATED;<a name="line.610"></a> |
| <span class="sourceLineNo">611</span> return strParam.subSequence(0, truncatedLength) + truncatedFlag;<a name="line.611"></a> |
| <span class="sourceLineNo">612</span> }<a name="line.612"></a> |
| <span class="sourceLineNo">613</span> return strParam.subSequence(0, 150) + KEY_WORD_TRUNCATED;<a name="line.613"></a> |
| <span class="sourceLineNo">614</span> }<a name="line.614"></a> |
| <span class="sourceLineNo">615</span><a name="line.615"></a> |
| <span class="sourceLineNo">616</span> /**<a name="line.616"></a> |
| <span class="sourceLineNo">617</span> * Set the handler for calling out of RPC for error conditions.<a name="line.617"></a> |
| <span class="sourceLineNo">618</span> * @param handler the handler implementation<a name="line.618"></a> |
| <span class="sourceLineNo">619</span> */<a name="line.619"></a> |
| <span class="sourceLineNo">620</span> @Override<a name="line.620"></a> |
| <span class="sourceLineNo">621</span> public void setErrorHandler(HBaseRPCErrorHandler handler) {<a name="line.621"></a> |
| <span class="sourceLineNo">622</span> this.errorHandler = handler;<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> @Override<a name="line.625"></a> |
| <span class="sourceLineNo">626</span> public HBaseRPCErrorHandler getErrorHandler() {<a name="line.626"></a> |
| <span class="sourceLineNo">627</span> return this.errorHandler;<a name="line.627"></a> |
| <span class="sourceLineNo">628</span> }<a name="line.628"></a> |
| <span class="sourceLineNo">629</span><a name="line.629"></a> |
| <span class="sourceLineNo">630</span> /**<a name="line.630"></a> |
| <span class="sourceLineNo">631</span> * Returns the metrics instance for reporting RPC call statistics<a name="line.631"></a> |
| <span class="sourceLineNo">632</span> */<a name="line.632"></a> |
| <span class="sourceLineNo">633</span> @Override<a name="line.633"></a> |
| <span class="sourceLineNo">634</span> public MetricsHBaseServer getMetrics() {<a name="line.634"></a> |
| <span class="sourceLineNo">635</span> return metrics;<a name="line.635"></a> |
| <span class="sourceLineNo">636</span> }<a name="line.636"></a> |
| <span class="sourceLineNo">637</span><a name="line.637"></a> |
| <span class="sourceLineNo">638</span> @Override<a name="line.638"></a> |
| <span class="sourceLineNo">639</span> public void addCallSize(final long diff) {<a name="line.639"></a> |
| <span class="sourceLineNo">640</span> this.callQueueSizeInBytes.add(diff);<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> /**<a name="line.643"></a> |
| <span class="sourceLineNo">644</span> * Authorize the incoming client connection.<a name="line.644"></a> |
| <span class="sourceLineNo">645</span> * @param user client user<a name="line.645"></a> |
| <span class="sourceLineNo">646</span> * @param connection incoming connection<a name="line.646"></a> |
| <span class="sourceLineNo">647</span> * @param addr InetAddress of incoming connection<a name="line.647"></a> |
| <span class="sourceLineNo">648</span> * @throws AuthorizationException when the client isn't authorized to talk the protocol<a name="line.648"></a> |
| <span class="sourceLineNo">649</span> */<a name="line.649"></a> |
| <span class="sourceLineNo">650</span> public synchronized void authorize(UserGroupInformation user, ConnectionHeader connection,<a name="line.650"></a> |
| <span class="sourceLineNo">651</span> InetAddress addr) throws AuthorizationException {<a name="line.651"></a> |
| <span class="sourceLineNo">652</span> if (authorize) {<a name="line.652"></a> |
| <span class="sourceLineNo">653</span> Class<?> c = getServiceInterface(services, connection.getServiceName());<a name="line.653"></a> |
| <span class="sourceLineNo">654</span> authManager.authorize(user, c, getConf(), addr);<a name="line.654"></a> |
| <span class="sourceLineNo">655</span> }<a name="line.655"></a> |
| <span class="sourceLineNo">656</span> }<a name="line.656"></a> |
| <span class="sourceLineNo">657</span><a name="line.657"></a> |
| <span class="sourceLineNo">658</span> /**<a name="line.658"></a> |
| <span class="sourceLineNo">659</span> * When the read or write buffer size is larger than this limit, i/o will be done in chunks of<a name="line.659"></a> |
| <span class="sourceLineNo">660</span> * this size. Most RPC requests and responses would be be smaller.<a name="line.660"></a> |
| <span class="sourceLineNo">661</span> */<a name="line.661"></a> |
| <span class="sourceLineNo">662</span> protected static final int NIO_BUFFER_LIMIT = 64 * 1024; // should not be more than 64KB.<a name="line.662"></a> |
| <span class="sourceLineNo">663</span><a name="line.663"></a> |
| <span class="sourceLineNo">664</span> /**<a name="line.664"></a> |
| <span class="sourceLineNo">665</span> * This is a wrapper around<a name="line.665"></a> |
| <span class="sourceLineNo">666</span> * {@link java.nio.channels.ReadableByteChannel#read(java.nio.ByteBuffer)}. If the amount of data<a name="line.666"></a> |
| <span class="sourceLineNo">667</span> * is large, it writes to channel in smaller chunks. This is to avoid jdk from creating many<a name="line.667"></a> |
| <span class="sourceLineNo">668</span> * direct buffers as the size of ByteBuffer increases. There should not be any performance<a name="line.668"></a> |
| <span class="sourceLineNo">669</span> * degredation.<a name="line.669"></a> |
| <span class="sourceLineNo">670</span> * @param channel writable byte channel to write on<a name="line.670"></a> |
| <span class="sourceLineNo">671</span> * @param buffer buffer to write<a name="line.671"></a> |
| <span class="sourceLineNo">672</span> * @return number of bytes written<a name="line.672"></a> |
| <span class="sourceLineNo">673</span> * @throws java.io.IOException e<a name="line.673"></a> |
| <span class="sourceLineNo">674</span> * @see java.nio.channels.ReadableByteChannel#read(java.nio.ByteBuffer)<a name="line.674"></a> |
| <span class="sourceLineNo">675</span> */<a name="line.675"></a> |
| <span class="sourceLineNo">676</span> protected int channelRead(ReadableByteChannel channel, ByteBuffer buffer) throws IOException {<a name="line.676"></a> |
| <span class="sourceLineNo">677</span><a name="line.677"></a> |
| <span class="sourceLineNo">678</span> int count = (buffer.remaining() <= NIO_BUFFER_LIMIT)<a name="line.678"></a> |
| <span class="sourceLineNo">679</span> ? channel.read(buffer)<a name="line.679"></a> |
| <span class="sourceLineNo">680</span> : channelIO(channel, null, buffer);<a name="line.680"></a> |
| <span class="sourceLineNo">681</span> if (count > 0) {<a name="line.681"></a> |
| <span class="sourceLineNo">682</span> metrics.receivedBytes(count);<a name="line.682"></a> |
| <span class="sourceLineNo">683</span> }<a name="line.683"></a> |
| <span class="sourceLineNo">684</span> return count;<a name="line.684"></a> |
| <span class="sourceLineNo">685</span> }<a name="line.685"></a> |
| <span class="sourceLineNo">686</span><a name="line.686"></a> |
| <span class="sourceLineNo">687</span> /**<a name="line.687"></a> |
| <span class="sourceLineNo">688</span> * Helper for {@link #channelRead(java.nio.channels.ReadableByteChannel, java.nio.ByteBuffer)}.<a name="line.688"></a> |
| <span class="sourceLineNo">689</span> * Only one of readCh or writeCh should be non-null.<a name="line.689"></a> |
| <span class="sourceLineNo">690</span> * @param readCh read channel<a name="line.690"></a> |
| <span class="sourceLineNo">691</span> * @param writeCh write channel<a name="line.691"></a> |
| <span class="sourceLineNo">692</span> * @param buf buffer to read or write into/out of<a name="line.692"></a> |
| <span class="sourceLineNo">693</span> * @return bytes written<a name="line.693"></a> |
| <span class="sourceLineNo">694</span> * @throws java.io.IOException e<a name="line.694"></a> |
| <span class="sourceLineNo">695</span> * @see #channelRead(java.nio.channels.ReadableByteChannel, java.nio.ByteBuffer)<a name="line.695"></a> |
| <span class="sourceLineNo">696</span> */<a name="line.696"></a> |
| <span class="sourceLineNo">697</span> private static int channelIO(ReadableByteChannel readCh, WritableByteChannel writeCh,<a name="line.697"></a> |
| <span class="sourceLineNo">698</span> ByteBuffer buf) throws IOException {<a name="line.698"></a> |
| <span class="sourceLineNo">699</span><a name="line.699"></a> |
| <span class="sourceLineNo">700</span> int originalLimit = buf.limit();<a name="line.700"></a> |
| <span class="sourceLineNo">701</span> int initialRemaining = buf.remaining();<a name="line.701"></a> |
| <span class="sourceLineNo">702</span> int ret = 0;<a name="line.702"></a> |
| <span class="sourceLineNo">703</span><a name="line.703"></a> |
| <span class="sourceLineNo">704</span> while (buf.remaining() > 0) {<a name="line.704"></a> |
| <span class="sourceLineNo">705</span> try {<a name="line.705"></a> |
| <span class="sourceLineNo">706</span> int ioSize = Math.min(buf.remaining(), NIO_BUFFER_LIMIT);<a name="line.706"></a> |
| <span class="sourceLineNo">707</span> buf.limit(buf.position() + ioSize);<a name="line.707"></a> |
| <span class="sourceLineNo">708</span><a name="line.708"></a> |
| <span class="sourceLineNo">709</span> ret = (readCh == null) ? writeCh.write(buf) : readCh.read(buf);<a name="line.709"></a> |
| <span class="sourceLineNo">710</span><a name="line.710"></a> |
| <span class="sourceLineNo">711</span> if (ret < ioSize) {<a name="line.711"></a> |
| <span class="sourceLineNo">712</span> break;<a name="line.712"></a> |
| <span class="sourceLineNo">713</span> }<a name="line.713"></a> |
| <span class="sourceLineNo">714</span><a name="line.714"></a> |
| <span class="sourceLineNo">715</span> } finally {<a name="line.715"></a> |
| <span class="sourceLineNo">716</span> buf.limit(originalLimit);<a name="line.716"></a> |
| <span class="sourceLineNo">717</span> }<a name="line.717"></a> |
| <span class="sourceLineNo">718</span> }<a name="line.718"></a> |
| <span class="sourceLineNo">719</span><a name="line.719"></a> |
| <span class="sourceLineNo">720</span> int nBytes = initialRemaining - buf.remaining();<a name="line.720"></a> |
| <span class="sourceLineNo">721</span> return (nBytes > 0) ? nBytes : ret;<a name="line.721"></a> |
| <span class="sourceLineNo">722</span> }<a name="line.722"></a> |
| <span class="sourceLineNo">723</span><a name="line.723"></a> |
| <span class="sourceLineNo">724</span> /**<a name="line.724"></a> |
| <span class="sourceLineNo">725</span> * Needed for features such as delayed calls. We need to be able to store the current call so that<a name="line.725"></a> |
| <span class="sourceLineNo">726</span> * we can complete it later or ask questions of what is supported by the current ongoing call.<a name="line.726"></a> |
| <span class="sourceLineNo">727</span> * @return An RpcCallContext backed by the currently ongoing call (gotten from a thread local)<a name="line.727"></a> |
| <span class="sourceLineNo">728</span> */<a name="line.728"></a> |
| <span class="sourceLineNo">729</span> public static Optional<RpcCall> getCurrentCall() {<a name="line.729"></a> |
| <span class="sourceLineNo">730</span> return Optional.ofNullable(CurCall.get());<a name="line.730"></a> |
| <span class="sourceLineNo">731</span> }<a name="line.731"></a> |
| <span class="sourceLineNo">732</span><a name="line.732"></a> |
| <span class="sourceLineNo">733</span> /**<a name="line.733"></a> |
| <span class="sourceLineNo">734</span> * Just return the current rpc call if it is a {@link ServerCall} and also has {@link CellScanner}<a name="line.734"></a> |
| <span class="sourceLineNo">735</span> * attached.<a name="line.735"></a> |
| <span class="sourceLineNo">736</span> * <p/><a name="line.736"></a> |
| <span class="sourceLineNo">737</span> * Mainly used for reference counting as {@link CellScanner} may reference non heap memory.<a name="line.737"></a> |
| <span class="sourceLineNo">738</span> */<a name="line.738"></a> |
| <span class="sourceLineNo">739</span> public static Optional<ServerCall<?>> getCurrentServerCallWithCellScanner() {<a name="line.739"></a> |
| <span class="sourceLineNo">740</span> return getCurrentCall().filter(c -> c instanceof ServerCall)<a name="line.740"></a> |
| <span class="sourceLineNo">741</span> .filter(c -> c.getCellScanner() != null).map(c -> (ServerCall<?>) c);<a name="line.741"></a> |
| <span class="sourceLineNo">742</span> }<a name="line.742"></a> |
| <span class="sourceLineNo">743</span><a name="line.743"></a> |
| <span class="sourceLineNo">744</span> public static boolean isInRpcCallContext() {<a name="line.744"></a> |
| <span class="sourceLineNo">745</span> return CurCall.get() != null;<a name="line.745"></a> |
| <span class="sourceLineNo">746</span> }<a name="line.746"></a> |
| <span class="sourceLineNo">747</span><a name="line.747"></a> |
| <span class="sourceLineNo">748</span> /**<a name="line.748"></a> |
| <span class="sourceLineNo">749</span> * Used by {@link org.apache.hadoop.hbase.procedure2.store.region.RegionProcedureStore}. For<a name="line.749"></a> |
| <span class="sourceLineNo">750</span> * master's rpc call, it may generate new procedure and mutate the region which store procedure.<a name="line.750"></a> |
| <span class="sourceLineNo">751</span> * There are some check about rpc when mutate region, such as rpc timeout check. So unset the rpc<a name="line.751"></a> |
| <span class="sourceLineNo">752</span> * call to avoid the rpc check.<a name="line.752"></a> |
| <span class="sourceLineNo">753</span> * @return the currently ongoing rpc call<a name="line.753"></a> |
| <span class="sourceLineNo">754</span> */<a name="line.754"></a> |
| <span class="sourceLineNo">755</span> public static Optional<RpcCall> unsetCurrentCall() {<a name="line.755"></a> |
| <span class="sourceLineNo">756</span> Optional<RpcCall> rpcCall = getCurrentCall();<a name="line.756"></a> |
| <span class="sourceLineNo">757</span> CurCall.set(null);<a name="line.757"></a> |
| <span class="sourceLineNo">758</span> return rpcCall;<a name="line.758"></a> |
| <span class="sourceLineNo">759</span> }<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> * Used by {@link org.apache.hadoop.hbase.procedure2.store.region.RegionProcedureStore}. Set the<a name="line.762"></a> |
| <span class="sourceLineNo">763</span> * rpc call back after mutate region.<a name="line.763"></a> |
| <span class="sourceLineNo">764</span> */<a name="line.764"></a> |
| <span class="sourceLineNo">765</span> public static void setCurrentCall(RpcCall rpcCall) {<a name="line.765"></a> |
| <span class="sourceLineNo">766</span> CurCall.set(rpcCall);<a name="line.766"></a> |
| <span class="sourceLineNo">767</span> }<a name="line.767"></a> |
| <span class="sourceLineNo">768</span><a name="line.768"></a> |
| <span class="sourceLineNo">769</span> /**<a name="line.769"></a> |
| <span class="sourceLineNo">770</span> * Returns the user credentials associated with the current RPC request or not present if no<a name="line.770"></a> |
| <span class="sourceLineNo">771</span> * credentials were provided.<a name="line.771"></a> |
| <span class="sourceLineNo">772</span> * @return A User<a name="line.772"></a> |
| <span class="sourceLineNo">773</span> */<a name="line.773"></a> |
| <span class="sourceLineNo">774</span> public static Optional<User> getRequestUser() {<a name="line.774"></a> |
| <span class="sourceLineNo">775</span> Optional<RpcCall> ctx = getCurrentCall();<a name="line.775"></a> |
| <span class="sourceLineNo">776</span> return ctx.isPresent() ? ctx.get().getRequestUser() : Optional.empty();<a name="line.776"></a> |
| <span class="sourceLineNo">777</span> }<a name="line.777"></a> |
| <span class="sourceLineNo">778</span><a name="line.778"></a> |
| <span class="sourceLineNo">779</span> /**<a name="line.779"></a> |
| <span class="sourceLineNo">780</span> * The number of open RPC conections<a name="line.780"></a> |
| <span class="sourceLineNo">781</span> * @return the number of open rpc connections<a name="line.781"></a> |
| <span class="sourceLineNo">782</span> */<a name="line.782"></a> |
| <span class="sourceLineNo">783</span> abstract public int getNumOpenConnections();<a name="line.783"></a> |
| <span class="sourceLineNo">784</span><a name="line.784"></a> |
| <span class="sourceLineNo">785</span> /**<a name="line.785"></a> |
| <span class="sourceLineNo">786</span> * Returns the username for any user associated with the current RPC request or not present if no<a name="line.786"></a> |
| <span class="sourceLineNo">787</span> * user is set.<a name="line.787"></a> |
| <span class="sourceLineNo">788</span> */<a name="line.788"></a> |
| <span class="sourceLineNo">789</span> public static Optional<String> getRequestUserName() {<a name="line.789"></a> |
| <span class="sourceLineNo">790</span> return getRequestUser().map(User::getShortName);<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> /** Returns Address of remote client if a request is ongoing, else null */<a name="line.793"></a> |
| <span class="sourceLineNo">794</span> public static Optional<InetAddress> getRemoteAddress() {<a name="line.794"></a> |
| <span class="sourceLineNo">795</span> return getCurrentCall().map(RpcCall::getRemoteAddress);<a name="line.795"></a> |
| <span class="sourceLineNo">796</span> }<a name="line.796"></a> |
| <span class="sourceLineNo">797</span><a name="line.797"></a> |
| <span class="sourceLineNo">798</span> /**<a name="line.798"></a> |
| <span class="sourceLineNo">799</span> * @param serviceName Some arbitrary string that represents a 'service'.<a name="line.799"></a> |
| <span class="sourceLineNo">800</span> * @param services Available service instances<a name="line.800"></a> |
| <span class="sourceLineNo">801</span> * @return Matching BlockingServiceAndInterface pair<a name="line.801"></a> |
| <span class="sourceLineNo">802</span> */<a name="line.802"></a> |
| <span class="sourceLineNo">803</span> protected static BlockingServiceAndInterface getServiceAndInterface(<a name="line.803"></a> |
| <span class="sourceLineNo">804</span> final List<BlockingServiceAndInterface> services, final String serviceName) {<a name="line.804"></a> |
| <span class="sourceLineNo">805</span> for (BlockingServiceAndInterface bs : services) {<a name="line.805"></a> |
| <span class="sourceLineNo">806</span> if (bs.getBlockingService().getDescriptorForType().getName().equals(serviceName)) {<a name="line.806"></a> |
| <span class="sourceLineNo">807</span> return bs;<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> return null;<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> /**<a name="line.813"></a> |
| <span class="sourceLineNo">814</span> * @param serviceName Some arbitrary string that represents a 'service'.<a name="line.814"></a> |
| <span class="sourceLineNo">815</span> * @param services Available services and their service interfaces.<a name="line.815"></a> |
| <span class="sourceLineNo">816</span> * @return Service interface class for <code>serviceName</code><a name="line.816"></a> |
| <span class="sourceLineNo">817</span> */<a name="line.817"></a> |
| <span class="sourceLineNo">818</span> protected static Class<?> getServiceInterface(final List<BlockingServiceAndInterface> services,<a name="line.818"></a> |
| <span class="sourceLineNo">819</span> final String serviceName) {<a name="line.819"></a> |
| <span class="sourceLineNo">820</span> BlockingServiceAndInterface bsasi = getServiceAndInterface(services, serviceName);<a name="line.820"></a> |
| <span class="sourceLineNo">821</span> return bsasi == null ? null : bsasi.getServiceInterface();<a name="line.821"></a> |
| <span class="sourceLineNo">822</span> }<a name="line.822"></a> |
| <span class="sourceLineNo">823</span><a name="line.823"></a> |
| <span class="sourceLineNo">824</span> /**<a name="line.824"></a> |
| <span class="sourceLineNo">825</span> * @param serviceName Some arbitrary string that represents a 'service'.<a name="line.825"></a> |
| <span class="sourceLineNo">826</span> * @param services Available services and their service interfaces.<a name="line.826"></a> |
| <span class="sourceLineNo">827</span> * @return BlockingService that goes with the passed <code>serviceName</code><a name="line.827"></a> |
| <span class="sourceLineNo">828</span> */<a name="line.828"></a> |
| <span class="sourceLineNo">829</span> protected static BlockingService getService(final List<BlockingServiceAndInterface> services,<a name="line.829"></a> |
| <span class="sourceLineNo">830</span> final String serviceName) {<a name="line.830"></a> |
| <span class="sourceLineNo">831</span> BlockingServiceAndInterface bsasi = getServiceAndInterface(services, serviceName);<a name="line.831"></a> |
| <span class="sourceLineNo">832</span> return bsasi == null ? null : bsasi.getBlockingService();<a name="line.832"></a> |
| <span class="sourceLineNo">833</span> }<a name="line.833"></a> |
| <span class="sourceLineNo">834</span><a name="line.834"></a> |
| <span class="sourceLineNo">835</span> protected static MonitoredRPCHandler getStatus() {<a name="line.835"></a> |
| <span class="sourceLineNo">836</span> // It is ugly the way we park status up in RpcServer. Let it be for now. TODO.<a name="line.836"></a> |
| <span class="sourceLineNo">837</span> MonitoredRPCHandler status = RpcServer.MONITORED_RPC.get();<a name="line.837"></a> |
| <span class="sourceLineNo">838</span> if (status != null) {<a name="line.838"></a> |
| <span class="sourceLineNo">839</span> return status;<a name="line.839"></a> |
| <span class="sourceLineNo">840</span> }<a name="line.840"></a> |
| <span class="sourceLineNo">841</span> status = TaskMonitor.get().createRPCStatus(Thread.currentThread().getName());<a name="line.841"></a> |
| <span class="sourceLineNo">842</span> status.pause("Waiting for a call");<a name="line.842"></a> |
| <span class="sourceLineNo">843</span> RpcServer.MONITORED_RPC.set(status);<a name="line.843"></a> |
| <span class="sourceLineNo">844</span> return status;<a name="line.844"></a> |
| <span class="sourceLineNo">845</span> }<a name="line.845"></a> |
| <span class="sourceLineNo">846</span><a name="line.846"></a> |
| <span class="sourceLineNo">847</span> /**<a name="line.847"></a> |
| <span class="sourceLineNo">848</span> * Returns the remote side ip address when invoked inside an RPC Returns null incase of an error.<a name="line.848"></a> |
| <span class="sourceLineNo">849</span> */<a name="line.849"></a> |
| <span class="sourceLineNo">850</span> public static InetAddress getRemoteIp() {<a name="line.850"></a> |
| <span class="sourceLineNo">851</span> RpcCall call = CurCall.get();<a name="line.851"></a> |
| <span class="sourceLineNo">852</span> if (call != null) {<a name="line.852"></a> |
| <span class="sourceLineNo">853</span> return call.getRemoteAddress();<a name="line.853"></a> |
| <span class="sourceLineNo">854</span> }<a name="line.854"></a> |
| <span class="sourceLineNo">855</span> return null;<a name="line.855"></a> |
| <span class="sourceLineNo">856</span> }<a name="line.856"></a> |
| <span class="sourceLineNo">857</span><a name="line.857"></a> |
| <span class="sourceLineNo">858</span> @Override<a name="line.858"></a> |
| <span class="sourceLineNo">859</span> public RpcScheduler getScheduler() {<a name="line.859"></a> |
| <span class="sourceLineNo">860</span> return scheduler;<a name="line.860"></a> |
| <span class="sourceLineNo">861</span> }<a name="line.861"></a> |
| <span class="sourceLineNo">862</span><a name="line.862"></a> |
| <span class="sourceLineNo">863</span> @Override<a name="line.863"></a> |
| <span class="sourceLineNo">864</span> public ByteBuffAllocator getByteBuffAllocator() {<a name="line.864"></a> |
| <span class="sourceLineNo">865</span> return this.bbAllocator;<a name="line.865"></a> |
| <span class="sourceLineNo">866</span> }<a name="line.866"></a> |
| <span class="sourceLineNo">867</span><a name="line.867"></a> |
| <span class="sourceLineNo">868</span> @Override<a name="line.868"></a> |
| <span class="sourceLineNo">869</span> public void setRsRpcServices(RSRpcServices rsRpcServices) {<a name="line.869"></a> |
| <span class="sourceLineNo">870</span> this.rsRpcServices = rsRpcServices;<a name="line.870"></a> |
| <span class="sourceLineNo">871</span> }<a name="line.871"></a> |
| <span class="sourceLineNo">872</span><a name="line.872"></a> |
| <span class="sourceLineNo">873</span> @Override<a name="line.873"></a> |
| <span class="sourceLineNo">874</span> public void setNamedQueueRecorder(NamedQueueRecorder namedQueueRecorder) {<a name="line.874"></a> |
| <span class="sourceLineNo">875</span> this.namedQueueRecorder = namedQueueRecorder;<a name="line.875"></a> |
| <span class="sourceLineNo">876</span> }<a name="line.876"></a> |
| <span class="sourceLineNo">877</span><a name="line.877"></a> |
| <span class="sourceLineNo">878</span> protected boolean needAuthorization() {<a name="line.878"></a> |
| <span class="sourceLineNo">879</span> return authorize;<a name="line.879"></a> |
| <span class="sourceLineNo">880</span> }<a name="line.880"></a> |
| <span class="sourceLineNo">881</span>}<a name="line.881"></a> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| </pre> |
| </div> |
| </body> |
| </html> |