| <!DOCTYPE HTML> |
| <html lang="en"> |
| <head> |
| <!-- Generated by javadoc (17) --> |
| <title>Source code</title> |
| <meta name="viewport" content="width=device-width, initial-scale=1"> |
| <meta name="description" content="source: package: org.apache.hadoop.hbase.ipc, class: SimpleServerRpcConnection"> |
| <meta name="generator" content="javadoc/SourceToHTMLConverter"> |
| <link rel="stylesheet" type="text/css" href="../../../../../../stylesheet.css" title="Style"> |
| </head> |
| <body class="source-page"> |
| <main role="main"> |
| <div class="source-container"> |
| <pre><span class="source-line-no">001</span><span id="line-1">/*</span> |
| <span class="source-line-no">002</span><span id="line-2"> * Licensed to the Apache Software Foundation (ASF) under one</span> |
| <span class="source-line-no">003</span><span id="line-3"> * or more contributor license agreements. See the NOTICE file</span> |
| <span class="source-line-no">004</span><span id="line-4"> * distributed with this work for additional information</span> |
| <span class="source-line-no">005</span><span id="line-5"> * regarding copyright ownership. The ASF licenses this file</span> |
| <span class="source-line-no">006</span><span id="line-6"> * to you under the Apache License, Version 2.0 (the</span> |
| <span class="source-line-no">007</span><span id="line-7"> * "License"); you may not use this file except in compliance</span> |
| <span class="source-line-no">008</span><span id="line-8"> * with the License. You may obtain a copy of the License at</span> |
| <span class="source-line-no">009</span><span id="line-9"> *</span> |
| <span class="source-line-no">010</span><span id="line-10"> * http://www.apache.org/licenses/LICENSE-2.0</span> |
| <span class="source-line-no">011</span><span id="line-11"> *</span> |
| <span class="source-line-no">012</span><span id="line-12"> * Unless required by applicable law or agreed to in writing, software</span> |
| <span class="source-line-no">013</span><span id="line-13"> * distributed under the License is distributed on an "AS IS" BASIS,</span> |
| <span class="source-line-no">014</span><span id="line-14"> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span> |
| <span class="source-line-no">015</span><span id="line-15"> * See the License for the specific language governing permissions and</span> |
| <span class="source-line-no">016</span><span id="line-16"> * limitations under the License.</span> |
| <span class="source-line-no">017</span><span id="line-17"> */</span> |
| <span class="source-line-no">018</span><span id="line-18">package org.apache.hadoop.hbase.ipc;</span> |
| <span class="source-line-no">019</span><span id="line-19"></span> |
| <span class="source-line-no">020</span><span id="line-20">import java.io.ByteArrayInputStream;</span> |
| <span class="source-line-no">021</span><span id="line-21">import java.io.IOException;</span> |
| <span class="source-line-no">022</span><span id="line-22">import java.io.InputStream;</span> |
| <span class="source-line-no">023</span><span id="line-23">import java.net.InetAddress;</span> |
| <span class="source-line-no">024</span><span id="line-24">import java.net.Socket;</span> |
| <span class="source-line-no">025</span><span id="line-25">import java.nio.ByteBuffer;</span> |
| <span class="source-line-no">026</span><span id="line-26">import java.nio.channels.Channels;</span> |
| <span class="source-line-no">027</span><span id="line-27">import java.nio.channels.ReadableByteChannel;</span> |
| <span class="source-line-no">028</span><span id="line-28">import java.nio.channels.SocketChannel;</span> |
| <span class="source-line-no">029</span><span id="line-29">import java.util.concurrent.ConcurrentLinkedDeque;</span> |
| <span class="source-line-no">030</span><span id="line-30">import java.util.concurrent.atomic.LongAdder;</span> |
| <span class="source-line-no">031</span><span id="line-31">import java.util.concurrent.locks.Lock;</span> |
| <span class="source-line-no">032</span><span id="line-32">import java.util.concurrent.locks.ReentrantLock;</span> |
| <span class="source-line-no">033</span><span id="line-33">import org.apache.hadoop.hbase.DoNotRetryIOException;</span> |
| <span class="source-line-no">034</span><span id="line-34">import org.apache.hadoop.hbase.ExtendedCellScanner;</span> |
| <span class="source-line-no">035</span><span id="line-35">import org.apache.hadoop.hbase.client.VersionInfoUtil;</span> |
| <span class="source-line-no">036</span><span id="line-36">import org.apache.hadoop.hbase.exceptions.RequestTooBigException;</span> |
| <span class="source-line-no">037</span><span id="line-37">import org.apache.hadoop.hbase.ipc.RpcServer.CallCleanup;</span> |
| <span class="source-line-no">038</span><span id="line-38">import org.apache.hadoop.hbase.nio.ByteBuff;</span> |
| <span class="source-line-no">039</span><span id="line-39">import org.apache.hadoop.hbase.nio.SingleByteBuff;</span> |
| <span class="source-line-no">040</span><span id="line-40">import org.apache.hadoop.hbase.security.HBaseSaslRpcServer;</span> |
| <span class="source-line-no">041</span><span id="line-41">import org.apache.hadoop.hbase.security.SaslStatus;</span> |
| <span class="source-line-no">042</span><span id="line-42">import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;</span> |
| <span class="source-line-no">043</span><span id="line-43">import org.apache.hadoop.io.BytesWritable;</span> |
| <span class="source-line-no">044</span><span id="line-44">import org.apache.yetus.audience.InterfaceAudience;</span> |
| <span class="source-line-no">045</span><span id="line-45"></span> |
| <span class="source-line-no">046</span><span id="line-46">import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;</span> |
| <span class="source-line-no">047</span><span id="line-47">import org.apache.hbase.thirdparty.com.google.protobuf.CodedInputStream;</span> |
| <span class="source-line-no">048</span><span id="line-48">import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;</span> |
| <span class="source-line-no">049</span><span id="line-49">import org.apache.hbase.thirdparty.com.google.protobuf.Message;</span> |
| <span class="source-line-no">050</span><span id="line-50"></span> |
| <span class="source-line-no">051</span><span id="line-51">import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;</span> |
| <span class="source-line-no">052</span><span id="line-52">import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader;</span> |
| <span class="source-line-no">053</span><span id="line-53"></span> |
| <span class="source-line-no">054</span><span id="line-54">/** Reads calls from a connection and queues them for handling. */</span> |
| <span class="source-line-no">055</span><span id="line-55">@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "VO_VOLATILE_INCREMENT",</span> |
| <span class="source-line-no">056</span><span id="line-56"> justification = "False positive according to http://sourceforge.net/p/findbugs/bugs/1032/")</span> |
| <span class="source-line-no">057</span><span id="line-57">@Deprecated</span> |
| <span class="source-line-no">058</span><span id="line-58">@InterfaceAudience.Private</span> |
| <span class="source-line-no">059</span><span id="line-59">class SimpleServerRpcConnection extends ServerRpcConnection {</span> |
| <span class="source-line-no">060</span><span id="line-60"></span> |
| <span class="source-line-no">061</span><span id="line-61"> final SocketChannel channel;</span> |
| <span class="source-line-no">062</span><span id="line-62"> private ByteBuff data;</span> |
| <span class="source-line-no">063</span><span id="line-63"> private ByteBuffer dataLengthBuffer;</span> |
| <span class="source-line-no">064</span><span id="line-64"> private ByteBuffer preambleBuffer;</span> |
| <span class="source-line-no">065</span><span id="line-65"> private final LongAdder rpcCount = new LongAdder(); // number of outstanding rpcs</span> |
| <span class="source-line-no">066</span><span id="line-66"> private long lastContact;</span> |
| <span class="source-line-no">067</span><span id="line-67"> private final Socket socket;</span> |
| <span class="source-line-no">068</span><span id="line-68"> final SimpleRpcServerResponder responder;</span> |
| <span class="source-line-no">069</span><span id="line-69"></span> |
| <span class="source-line-no">070</span><span id="line-70"> // If initial preamble with version and magic has been read or not.</span> |
| <span class="source-line-no">071</span><span id="line-71"> private boolean connectionPreambleRead = false;</span> |
| <span class="source-line-no">072</span><span id="line-72"> private boolean saslContextEstablished;</span> |
| <span class="source-line-no">073</span><span id="line-73"> private ByteBuffer unwrappedData;</span> |
| <span class="source-line-no">074</span><span id="line-74"> // When is this set? FindBugs wants to know! Says NP</span> |
| <span class="source-line-no">075</span><span id="line-75"> private ByteBuffer unwrappedDataLengthBuffer = ByteBuffer.allocate(4);</span> |
| <span class="source-line-no">076</span><span id="line-76"> boolean useWrap = false;</span> |
| <span class="source-line-no">077</span><span id="line-77"></span> |
| <span class="source-line-no">078</span><span id="line-78"> final ConcurrentLinkedDeque<RpcResponse> responseQueue = new ConcurrentLinkedDeque<>();</span> |
| <span class="source-line-no">079</span><span id="line-79"> final Lock responseWriteLock = new ReentrantLock();</span> |
| <span class="source-line-no">080</span><span id="line-80"> long lastSentTime = -1L;</span> |
| <span class="source-line-no">081</span><span id="line-81"></span> |
| <span class="source-line-no">082</span><span id="line-82"> public SimpleServerRpcConnection(SimpleRpcServer rpcServer, SocketChannel channel,</span> |
| <span class="source-line-no">083</span><span id="line-83"> long lastContact) {</span> |
| <span class="source-line-no">084</span><span id="line-84"> super(rpcServer);</span> |
| <span class="source-line-no">085</span><span id="line-85"> this.channel = channel;</span> |
| <span class="source-line-no">086</span><span id="line-86"> this.lastContact = lastContact;</span> |
| <span class="source-line-no">087</span><span id="line-87"> this.data = null;</span> |
| <span class="source-line-no">088</span><span id="line-88"> this.dataLengthBuffer = ByteBuffer.allocate(4);</span> |
| <span class="source-line-no">089</span><span id="line-89"> this.socket = channel.socket();</span> |
| <span class="source-line-no">090</span><span id="line-90"> this.addr = socket.getInetAddress();</span> |
| <span class="source-line-no">091</span><span id="line-91"> if (addr == null) {</span> |
| <span class="source-line-no">092</span><span id="line-92"> this.hostAddress = "*Unknown*";</span> |
| <span class="source-line-no">093</span><span id="line-93"> } else {</span> |
| <span class="source-line-no">094</span><span id="line-94"> this.hostAddress = addr.getHostAddress();</span> |
| <span class="source-line-no">095</span><span id="line-95"> }</span> |
| <span class="source-line-no">096</span><span id="line-96"> this.remotePort = socket.getPort();</span> |
| <span class="source-line-no">097</span><span id="line-97"> if (rpcServer.socketSendBufferSize != 0) {</span> |
| <span class="source-line-no">098</span><span id="line-98"> try {</span> |
| <span class="source-line-no">099</span><span id="line-99"> socket.setSendBufferSize(rpcServer.socketSendBufferSize);</span> |
| <span class="source-line-no">100</span><span id="line-100"> } catch (IOException e) {</span> |
| <span class="source-line-no">101</span><span id="line-101"> SimpleRpcServer.LOG.warn(</span> |
| <span class="source-line-no">102</span><span id="line-102"> "Connection: unable to set socket send buffer size to " + rpcServer.socketSendBufferSize);</span> |
| <span class="source-line-no">103</span><span id="line-103"> }</span> |
| <span class="source-line-no">104</span><span id="line-104"> }</span> |
| <span class="source-line-no">105</span><span id="line-105"> this.responder = rpcServer.responder;</span> |
| <span class="source-line-no">106</span><span id="line-106"> }</span> |
| <span class="source-line-no">107</span><span id="line-107"></span> |
| <span class="source-line-no">108</span><span id="line-108"> public void setLastContact(long lastContact) {</span> |
| <span class="source-line-no">109</span><span id="line-109"> this.lastContact = lastContact;</span> |
| <span class="source-line-no">110</span><span id="line-110"> }</span> |
| <span class="source-line-no">111</span><span id="line-111"></span> |
| <span class="source-line-no">112</span><span id="line-112"> public long getLastContact() {</span> |
| <span class="source-line-no">113</span><span id="line-113"> return lastContact;</span> |
| <span class="source-line-no">114</span><span id="line-114"> }</span> |
| <span class="source-line-no">115</span><span id="line-115"></span> |
| <span class="source-line-no">116</span><span id="line-116"> /* Return true if the connection has no outstanding rpc */</span> |
| <span class="source-line-no">117</span><span id="line-117"> boolean isIdle() {</span> |
| <span class="source-line-no">118</span><span id="line-118"> return rpcCount.sum() == 0;</span> |
| <span class="source-line-no">119</span><span id="line-119"> }</span> |
| <span class="source-line-no">120</span><span id="line-120"></span> |
| <span class="source-line-no">121</span><span id="line-121"> /* Decrement the outstanding RPC count */</span> |
| <span class="source-line-no">122</span><span id="line-122"> protected void decRpcCount() {</span> |
| <span class="source-line-no">123</span><span id="line-123"> rpcCount.decrement();</span> |
| <span class="source-line-no">124</span><span id="line-124"> }</span> |
| <span class="source-line-no">125</span><span id="line-125"></span> |
| <span class="source-line-no">126</span><span id="line-126"> /* Increment the outstanding RPC count */</span> |
| <span class="source-line-no">127</span><span id="line-127"> protected void incRpcCount() {</span> |
| <span class="source-line-no">128</span><span id="line-128"> rpcCount.increment();</span> |
| <span class="source-line-no">129</span><span id="line-129"> }</span> |
| <span class="source-line-no">130</span><span id="line-130"></span> |
| <span class="source-line-no">131</span><span id="line-131"> private int readPreamble() throws IOException {</span> |
| <span class="source-line-no">132</span><span id="line-132"> if (preambleBuffer == null) {</span> |
| <span class="source-line-no">133</span><span id="line-133"> preambleBuffer = ByteBuffer.allocate(6);</span> |
| <span class="source-line-no">134</span><span id="line-134"> }</span> |
| <span class="source-line-no">135</span><span id="line-135"> int count = this.rpcServer.channelRead(channel, preambleBuffer);</span> |
| <span class="source-line-no">136</span><span id="line-136"> if (count < 0 || preambleBuffer.remaining() > 0) {</span> |
| <span class="source-line-no">137</span><span id="line-137"> return count;</span> |
| <span class="source-line-no">138</span><span id="line-138"> }</span> |
| <span class="source-line-no">139</span><span id="line-139"> preambleBuffer.flip();</span> |
| <span class="source-line-no">140</span><span id="line-140"> PreambleResponse resp = processPreamble(preambleBuffer);</span> |
| <span class="source-line-no">141</span><span id="line-141"> switch (resp) {</span> |
| <span class="source-line-no">142</span><span id="line-142"> case SUCCEED:</span> |
| <span class="source-line-no">143</span><span id="line-143"> preambleBuffer = null; // do not need it anymore</span> |
| <span class="source-line-no">144</span><span id="line-144"> connectionPreambleRead = true;</span> |
| <span class="source-line-no">145</span><span id="line-145"> return count;</span> |
| <span class="source-line-no">146</span><span id="line-146"> case CONTINUE:</span> |
| <span class="source-line-no">147</span><span id="line-147"> // wait for the next preamble header</span> |
| <span class="source-line-no">148</span><span id="line-148"> preambleBuffer.clear();</span> |
| <span class="source-line-no">149</span><span id="line-149"> return count;</span> |
| <span class="source-line-no">150</span><span id="line-150"> case CLOSE:</span> |
| <span class="source-line-no">151</span><span id="line-151"> return -1;</span> |
| <span class="source-line-no">152</span><span id="line-152"> default:</span> |
| <span class="source-line-no">153</span><span id="line-153"> throw new IllegalArgumentException("Unknown preamble response: " + resp);</span> |
| <span class="source-line-no">154</span><span id="line-154"> }</span> |
| <span class="source-line-no">155</span><span id="line-155"> }</span> |
| <span class="source-line-no">156</span><span id="line-156"></span> |
| <span class="source-line-no">157</span><span id="line-157"> private int read4Bytes() throws IOException {</span> |
| <span class="source-line-no">158</span><span id="line-158"> if (this.dataLengthBuffer.remaining() > 0) {</span> |
| <span class="source-line-no">159</span><span id="line-159"> return this.rpcServer.channelRead(channel, this.dataLengthBuffer);</span> |
| <span class="source-line-no">160</span><span id="line-160"> } else {</span> |
| <span class="source-line-no">161</span><span id="line-161"> return 0;</span> |
| <span class="source-line-no">162</span><span id="line-162"> }</span> |
| <span class="source-line-no">163</span><span id="line-163"> }</span> |
| <span class="source-line-no">164</span><span id="line-164"></span> |
| <span class="source-line-no">165</span><span id="line-165"> private void processUnwrappedData(byte[] inBuf) throws IOException, InterruptedException {</span> |
| <span class="source-line-no">166</span><span id="line-166"> ReadableByteChannel ch = Channels.newChannel(new ByteArrayInputStream(inBuf));</span> |
| <span class="source-line-no">167</span><span id="line-167"> // Read all RPCs contained in the inBuf, even partial ones</span> |
| <span class="source-line-no">168</span><span id="line-168"> while (true) {</span> |
| <span class="source-line-no">169</span><span id="line-169"> int count;</span> |
| <span class="source-line-no">170</span><span id="line-170"> if (unwrappedDataLengthBuffer.remaining() > 0) {</span> |
| <span class="source-line-no">171</span><span id="line-171"> count = this.rpcServer.channelRead(ch, unwrappedDataLengthBuffer);</span> |
| <span class="source-line-no">172</span><span id="line-172"> if (count <= 0 || unwrappedDataLengthBuffer.remaining() > 0) {</span> |
| <span class="source-line-no">173</span><span id="line-173"> return;</span> |
| <span class="source-line-no">174</span><span id="line-174"> }</span> |
| <span class="source-line-no">175</span><span id="line-175"> }</span> |
| <span class="source-line-no">176</span><span id="line-176"></span> |
| <span class="source-line-no">177</span><span id="line-177"> if (unwrappedData == null) {</span> |
| <span class="source-line-no">178</span><span id="line-178"> unwrappedDataLengthBuffer.flip();</span> |
| <span class="source-line-no">179</span><span id="line-179"> int unwrappedDataLength = unwrappedDataLengthBuffer.getInt();</span> |
| <span class="source-line-no">180</span><span id="line-180"></span> |
| <span class="source-line-no">181</span><span id="line-181"> if (unwrappedDataLength == RpcClient.PING_CALL_ID) {</span> |
| <span class="source-line-no">182</span><span id="line-182"> if (RpcServer.LOG.isDebugEnabled()) RpcServer.LOG.debug("Received ping message");</span> |
| <span class="source-line-no">183</span><span id="line-183"> unwrappedDataLengthBuffer.clear();</span> |
| <span class="source-line-no">184</span><span id="line-184"> continue; // ping message</span> |
| <span class="source-line-no">185</span><span id="line-185"> }</span> |
| <span class="source-line-no">186</span><span id="line-186"> unwrappedData = ByteBuffer.allocate(unwrappedDataLength);</span> |
| <span class="source-line-no">187</span><span id="line-187"> }</span> |
| <span class="source-line-no">188</span><span id="line-188"></span> |
| <span class="source-line-no">189</span><span id="line-189"> count = this.rpcServer.channelRead(ch, unwrappedData);</span> |
| <span class="source-line-no">190</span><span id="line-190"> if (count <= 0 || unwrappedData.remaining() > 0) {</span> |
| <span class="source-line-no">191</span><span id="line-191"> return;</span> |
| <span class="source-line-no">192</span><span id="line-192"> }</span> |
| <span class="source-line-no">193</span><span id="line-193"></span> |
| <span class="source-line-no">194</span><span id="line-194"> if (unwrappedData.remaining() == 0) {</span> |
| <span class="source-line-no">195</span><span id="line-195"> unwrappedDataLengthBuffer.clear();</span> |
| <span class="source-line-no">196</span><span id="line-196"> unwrappedData.flip();</span> |
| <span class="source-line-no">197</span><span id="line-197"> processOneRpc(new SingleByteBuff(unwrappedData));</span> |
| <span class="source-line-no">198</span><span id="line-198"> unwrappedData = null;</span> |
| <span class="source-line-no">199</span><span id="line-199"> }</span> |
| <span class="source-line-no">200</span><span id="line-200"> }</span> |
| <span class="source-line-no">201</span><span id="line-201"> }</span> |
| <span class="source-line-no">202</span><span id="line-202"></span> |
| <span class="source-line-no">203</span><span id="line-203"> private void saslReadAndProcess(ByteBuff saslToken) throws IOException, InterruptedException {</span> |
| <span class="source-line-no">204</span><span id="line-204"> if (saslContextEstablished) {</span> |
| <span class="source-line-no">205</span><span id="line-205"> RpcServer.LOG.trace("Read input token of size={} for processing by saslServer.unwrap()",</span> |
| <span class="source-line-no">206</span><span id="line-206"> saslToken.limit());</span> |
| <span class="source-line-no">207</span><span id="line-207"> if (!useWrap) {</span> |
| <span class="source-line-no">208</span><span id="line-208"> processOneRpc(saslToken);</span> |
| <span class="source-line-no">209</span><span id="line-209"> } else {</span> |
| <span class="source-line-no">210</span><span id="line-210"> byte[] b = saslToken.hasArray() ? saslToken.array() : saslToken.toBytes();</span> |
| <span class="source-line-no">211</span><span id="line-211"> byte[] plaintextData = saslServer.unwrap(b, 0, b.length);</span> |
| <span class="source-line-no">212</span><span id="line-212"> // release the request buffer as we have already unwrapped all its content</span> |
| <span class="source-line-no">213</span><span id="line-213"> callCleanupIfNeeded();</span> |
| <span class="source-line-no">214</span><span id="line-214"> processUnwrappedData(plaintextData);</span> |
| <span class="source-line-no">215</span><span id="line-215"> }</span> |
| <span class="source-line-no">216</span><span id="line-216"> } else {</span> |
| <span class="source-line-no">217</span><span id="line-217"> byte[] replyToken;</span> |
| <span class="source-line-no">218</span><span id="line-218"> try {</span> |
| <span class="source-line-no">219</span><span id="line-219"> try {</span> |
| <span class="source-line-no">220</span><span id="line-220"> getOrCreateSaslServer();</span> |
| <span class="source-line-no">221</span><span id="line-221"> } catch (Exception e) {</span> |
| <span class="source-line-no">222</span><span id="line-222"> RpcServer.LOG.error("Error when trying to create instance of HBaseSaslRpcServer "</span> |
| <span class="source-line-no">223</span><span id="line-223"> + "with sasl provider: " + provider, e);</span> |
| <span class="source-line-no">224</span><span id="line-224"> throw e;</span> |
| <span class="source-line-no">225</span><span id="line-225"> }</span> |
| <span class="source-line-no">226</span><span id="line-226"> RpcServer.LOG.debug("Created SASL server with mechanism={}",</span> |
| <span class="source-line-no">227</span><span id="line-227"> provider.getSaslAuthMethod().getAuthMethod());</span> |
| <span class="source-line-no">228</span><span id="line-228"> RpcServer.LOG.debug(</span> |
| <span class="source-line-no">229</span><span id="line-229"> "Read input token of size={} for processing by saslServer." + "evaluateResponse()",</span> |
| <span class="source-line-no">230</span><span id="line-230"> saslToken.limit());</span> |
| <span class="source-line-no">231</span><span id="line-231"> replyToken = saslServer</span> |
| <span class="source-line-no">232</span><span id="line-232"> .evaluateResponse(saslToken.hasArray() ? saslToken.array() : saslToken.toBytes());</span> |
| <span class="source-line-no">233</span><span id="line-233"> } catch (IOException e) {</span> |
| <span class="source-line-no">234</span><span id="line-234"> RpcServer.LOG.debug("Failed to execute SASL handshake", e);</span> |
| <span class="source-line-no">235</span><span id="line-235"> Throwable sendToClient = HBaseSaslRpcServer.unwrap(e);</span> |
| <span class="source-line-no">236</span><span id="line-236"> doRawSaslReply(SaslStatus.ERROR, null, sendToClient.getClass().getName(),</span> |
| <span class="source-line-no">237</span><span id="line-237"> sendToClient.getLocalizedMessage());</span> |
| <span class="source-line-no">238</span><span id="line-238"> this.rpcServer.metrics.authenticationFailure();</span> |
| <span class="source-line-no">239</span><span id="line-239"> String clientIP = this.toString();</span> |
| <span class="source-line-no">240</span><span id="line-240"> // attempting user could be null</span> |
| <span class="source-line-no">241</span><span id="line-241"> RpcServer.AUDITLOG.warn("{}{}: {}", RpcServer.AUTH_FAILED_FOR, clientIP,</span> |
| <span class="source-line-no">242</span><span id="line-242"> saslServer.getAttemptingUser());</span> |
| <span class="source-line-no">243</span><span id="line-243"> throw e;</span> |
| <span class="source-line-no">244</span><span id="line-244"> } finally {</span> |
| <span class="source-line-no">245</span><span id="line-245"> // release the request buffer as we have already unwrapped all its content</span> |
| <span class="source-line-no">246</span><span id="line-246"> callCleanupIfNeeded();</span> |
| <span class="source-line-no">247</span><span id="line-247"> }</span> |
| <span class="source-line-no">248</span><span id="line-248"> if (replyToken != null) {</span> |
| <span class="source-line-no">249</span><span id="line-249"> if (RpcServer.LOG.isDebugEnabled()) {</span> |
| <span class="source-line-no">250</span><span id="line-250"> RpcServer.LOG.debug("Will send token of size " + replyToken.length + " from saslServer.");</span> |
| <span class="source-line-no">251</span><span id="line-251"> }</span> |
| <span class="source-line-no">252</span><span id="line-252"> doRawSaslReply(SaslStatus.SUCCESS, new BytesWritable(replyToken), null, null);</span> |
| <span class="source-line-no">253</span><span id="line-253"> }</span> |
| <span class="source-line-no">254</span><span id="line-254"> if (saslServer.isComplete()) {</span> |
| <span class="source-line-no">255</span><span id="line-255"> String qop = saslServer.getNegotiatedQop();</span> |
| <span class="source-line-no">256</span><span id="line-256"> useWrap = qop != null && !"auth".equalsIgnoreCase(qop);</span> |
| <span class="source-line-no">257</span><span id="line-257"> ugi =</span> |
| <span class="source-line-no">258</span><span id="line-258"> provider.getAuthorizedUgi(saslServer.getAuthorizationID(), this.rpcServer.secretManager);</span> |
| <span class="source-line-no">259</span><span id="line-259"> RpcServer.LOG.debug(</span> |
| <span class="source-line-no">260</span><span id="line-260"> "SASL server context established. Authenticated client: {}. Negotiated QoP is {}", ugi,</span> |
| <span class="source-line-no">261</span><span id="line-261"> qop);</span> |
| <span class="source-line-no">262</span><span id="line-262"> this.rpcServer.metrics.authenticationSuccess();</span> |
| <span class="source-line-no">263</span><span id="line-263"> RpcServer.AUDITLOG.info(RpcServer.AUTH_SUCCESSFUL_FOR + ugi);</span> |
| <span class="source-line-no">264</span><span id="line-264"> saslContextEstablished = true;</span> |
| <span class="source-line-no">265</span><span id="line-265"> }</span> |
| <span class="source-line-no">266</span><span id="line-266"> }</span> |
| <span class="source-line-no">267</span><span id="line-267"> }</span> |
| <span class="source-line-no">268</span><span id="line-268"></span> |
| <span class="source-line-no">269</span><span id="line-269"> /**</span> |
| <span class="source-line-no">270</span><span id="line-270"> * Read off the wire. If there is not enough data to read, update the connection state with what</span> |
| <span class="source-line-no">271</span><span id="line-271"> * we have and returns.</span> |
| <span class="source-line-no">272</span><span id="line-272"> * @return Returns -1 if failure (and caller will close connection), else zero or more.</span> |
| <span class="source-line-no">273</span><span id="line-273"> */</span> |
| <span class="source-line-no">274</span><span id="line-274"> public int readAndProcess() throws IOException, InterruptedException {</span> |
| <span class="source-line-no">275</span><span id="line-275"> // If we have not read the connection setup preamble, look to see if that is on the wire.</span> |
| <span class="source-line-no">276</span><span id="line-276"> if (!connectionPreambleRead) {</span> |
| <span class="source-line-no">277</span><span id="line-277"> int count = readPreamble();</span> |
| <span class="source-line-no">278</span><span id="line-278"> if (!connectionPreambleRead) {</span> |
| <span class="source-line-no">279</span><span id="line-279"> return count;</span> |
| <span class="source-line-no">280</span><span id="line-280"> }</span> |
| <span class="source-line-no">281</span><span id="line-281"> }</span> |
| <span class="source-line-no">282</span><span id="line-282"></span> |
| <span class="source-line-no">283</span><span id="line-283"> // Try and read in an int. it will be length of the data to read (or -1 if a ping). We catch the</span> |
| <span class="source-line-no">284</span><span id="line-284"> // integer length into the 4-byte this.dataLengthBuffer.</span> |
| <span class="source-line-no">285</span><span id="line-285"> int count = read4Bytes();</span> |
| <span class="source-line-no">286</span><span id="line-286"> if (count < 0 || dataLengthBuffer.remaining() > 0) {</span> |
| <span class="source-line-no">287</span><span id="line-287"> return count;</span> |
| <span class="source-line-no">288</span><span id="line-288"> }</span> |
| <span class="source-line-no">289</span><span id="line-289"></span> |
| <span class="source-line-no">290</span><span id="line-290"> // We have read a length and we have read the preamble. It is either the connection header</span> |
| <span class="source-line-no">291</span><span id="line-291"> // or it is a request.</span> |
| <span class="source-line-no">292</span><span id="line-292"> if (data == null) {</span> |
| <span class="source-line-no">293</span><span id="line-293"> dataLengthBuffer.flip();</span> |
| <span class="source-line-no">294</span><span id="line-294"> int dataLength = dataLengthBuffer.getInt();</span> |
| <span class="source-line-no">295</span><span id="line-295"> if (dataLength == RpcClient.PING_CALL_ID) {</span> |
| <span class="source-line-no">296</span><span id="line-296"> if (!useWrap) { // covers the !useSasl too</span> |
| <span class="source-line-no">297</span><span id="line-297"> dataLengthBuffer.clear();</span> |
| <span class="source-line-no">298</span><span id="line-298"> return 0; // ping message</span> |
| <span class="source-line-no">299</span><span id="line-299"> }</span> |
| <span class="source-line-no">300</span><span id="line-300"> }</span> |
| <span class="source-line-no">301</span><span id="line-301"> if (dataLength < 0) { // A data length of zero is legal.</span> |
| <span class="source-line-no">302</span><span id="line-302"> throw new DoNotRetryIOException(</span> |
| <span class="source-line-no">303</span><span id="line-303"> "Unexpected data length " + dataLength + "!! from " + getHostAddress());</span> |
| <span class="source-line-no">304</span><span id="line-304"> }</span> |
| <span class="source-line-no">305</span><span id="line-305"></span> |
| <span class="source-line-no">306</span><span id="line-306"> if (dataLength > this.rpcServer.maxRequestSize) {</span> |
| <span class="source-line-no">307</span><span id="line-307"> String msg = "RPC data length of " + dataLength + " received from " + getHostAddress()</span> |
| <span class="source-line-no">308</span><span id="line-308"> + " is greater than max allowed " + this.rpcServer.maxRequestSize + ". Set \""</span> |
| <span class="source-line-no">309</span><span id="line-309"> + SimpleRpcServer.MAX_REQUEST_SIZE</span> |
| <span class="source-line-no">310</span><span id="line-310"> + "\" on server to override this limit (not recommended)";</span> |
| <span class="source-line-no">311</span><span id="line-311"> SimpleRpcServer.LOG.warn(msg);</span> |
| <span class="source-line-no">312</span><span id="line-312"></span> |
| <span class="source-line-no">313</span><span id="line-313"> if (connectionHeaderRead && connectionPreambleRead) {</span> |
| <span class="source-line-no">314</span><span id="line-314"> incRpcCount();</span> |
| <span class="source-line-no">315</span><span id="line-315"> // Construct InputStream for the non-blocking SocketChannel</span> |
| <span class="source-line-no">316</span><span id="line-316"> // We need the InputStream because we want to read only the request header</span> |
| <span class="source-line-no">317</span><span id="line-317"> // instead of the whole rpc.</span> |
| <span class="source-line-no">318</span><span id="line-318"> ByteBuffer buf = ByteBuffer.allocate(1);</span> |
| <span class="source-line-no">319</span><span id="line-319"> InputStream is = new InputStream() {</span> |
| <span class="source-line-no">320</span><span id="line-320"> @Override</span> |
| <span class="source-line-no">321</span><span id="line-321"> public int read() throws IOException {</span> |
| <span class="source-line-no">322</span><span id="line-322"> SimpleServerRpcConnection.this.rpcServer.channelRead(channel, buf);</span> |
| <span class="source-line-no">323</span><span id="line-323"> buf.flip();</span> |
| <span class="source-line-no">324</span><span id="line-324"> int x = buf.get();</span> |
| <span class="source-line-no">325</span><span id="line-325"> buf.flip();</span> |
| <span class="source-line-no">326</span><span id="line-326"> return x;</span> |
| <span class="source-line-no">327</span><span id="line-327"> }</span> |
| <span class="source-line-no">328</span><span id="line-328"> };</span> |
| <span class="source-line-no">329</span><span id="line-329"> CodedInputStream cis = CodedInputStream.newInstance(is);</span> |
| <span class="source-line-no">330</span><span id="line-330"> int headerSize = cis.readRawVarint32();</span> |
| <span class="source-line-no">331</span><span id="line-331"> Message.Builder builder = RequestHeader.newBuilder();</span> |
| <span class="source-line-no">332</span><span id="line-332"> ProtobufUtil.mergeFrom(builder, cis, headerSize);</span> |
| <span class="source-line-no">333</span><span id="line-333"> RequestHeader header = (RequestHeader) builder.build();</span> |
| <span class="source-line-no">334</span><span id="line-334"></span> |
| <span class="source-line-no">335</span><span id="line-335"> // Notify the client about the offending request</span> |
| <span class="source-line-no">336</span><span id="line-336"> SimpleServerCall reqTooBig = new SimpleServerCall(header.getCallId(), this.service, null,</span> |
| <span class="source-line-no">337</span><span id="line-337"> null, null, null, this, 0, this.addr, EnvironmentEdgeManager.currentTime(), 0,</span> |
| <span class="source-line-no">338</span><span id="line-338"> this.rpcServer.bbAllocator, this.rpcServer.cellBlockBuilder, null, responder);</span> |
| <span class="source-line-no">339</span><span id="line-339"> RequestTooBigException reqTooBigEx = new RequestTooBigException(msg);</span> |
| <span class="source-line-no">340</span><span id="line-340"> this.rpcServer.metrics.exception(reqTooBigEx);</span> |
| <span class="source-line-no">341</span><span id="line-341"> // Make sure the client recognizes the underlying exception</span> |
| <span class="source-line-no">342</span><span id="line-342"> // Otherwise, throw a DoNotRetryIOException.</span> |
| <span class="source-line-no">343</span><span id="line-343"> if (</span> |
| <span class="source-line-no">344</span><span id="line-344"> VersionInfoUtil.hasMinimumVersion(connectionHeader.getVersionInfo(),</span> |
| <span class="source-line-no">345</span><span id="line-345"> RequestTooBigException.MAJOR_VERSION, RequestTooBigException.MINOR_VERSION)</span> |
| <span class="source-line-no">346</span><span id="line-346"> ) {</span> |
| <span class="source-line-no">347</span><span id="line-347"> reqTooBig.setResponse(null, null, reqTooBigEx, msg);</span> |
| <span class="source-line-no">348</span><span id="line-348"> } else {</span> |
| <span class="source-line-no">349</span><span id="line-349"> reqTooBig.setResponse(null, null, new DoNotRetryIOException(msg), msg);</span> |
| <span class="source-line-no">350</span><span id="line-350"> }</span> |
| <span class="source-line-no">351</span><span id="line-351"> // In most cases we will write out the response directly. If not, it is still OK to just</span> |
| <span class="source-line-no">352</span><span id="line-352"> // close the connection without writing out the reqTooBig response. Do not try to write</span> |
| <span class="source-line-no">353</span><span id="line-353"> // out directly here, and it will cause deserialization error if the connection is slow</span> |
| <span class="source-line-no">354</span><span id="line-354"> // and we have a half writing response in the queue.</span> |
| <span class="source-line-no">355</span><span id="line-355"> reqTooBig.sendResponseIfReady();</span> |
| <span class="source-line-no">356</span><span id="line-356"> }</span> |
| <span class="source-line-no">357</span><span id="line-357"> // Close the connection</span> |
| <span class="source-line-no">358</span><span id="line-358"> return -1;</span> |
| <span class="source-line-no">359</span><span id="line-359"> }</span> |
| <span class="source-line-no">360</span><span id="line-360"></span> |
| <span class="source-line-no">361</span><span id="line-361"> // Initialize this.data with a ByteBuff.</span> |
| <span class="source-line-no">362</span><span id="line-362"> // This call will allocate a ByteBuff to read request into and assign to this.data</span> |
| <span class="source-line-no">363</span><span id="line-363"> // Also when we use some buffer(s) from pool, it will create a CallCleanup instance also and</span> |
| <span class="source-line-no">364</span><span id="line-364"> // assign to this.callCleanup</span> |
| <span class="source-line-no">365</span><span id="line-365"> initByteBuffToReadInto(dataLength);</span> |
| <span class="source-line-no">366</span><span id="line-366"></span> |
| <span class="source-line-no">367</span><span id="line-367"> // Increment the rpc count. This counter will be decreased when we write</span> |
| <span class="source-line-no">368</span><span id="line-368"> // the response. If we want the connection to be detected as idle properly, we</span> |
| <span class="source-line-no">369</span><span id="line-369"> // need to keep the inc / dec correct.</span> |
| <span class="source-line-no">370</span><span id="line-370"> incRpcCount();</span> |
| <span class="source-line-no">371</span><span id="line-371"> }</span> |
| <span class="source-line-no">372</span><span id="line-372"></span> |
| <span class="source-line-no">373</span><span id="line-373"> count = channelDataRead(channel, data);</span> |
| <span class="source-line-no">374</span><span id="line-374"></span> |
| <span class="source-line-no">375</span><span id="line-375"> if (count >= 0 && data.remaining() == 0) { // count==0 if dataLength == 0</span> |
| <span class="source-line-no">376</span><span id="line-376"> process();</span> |
| <span class="source-line-no">377</span><span id="line-377"> }</span> |
| <span class="source-line-no">378</span><span id="line-378"></span> |
| <span class="source-line-no">379</span><span id="line-379"> return count;</span> |
| <span class="source-line-no">380</span><span id="line-380"> }</span> |
| <span class="source-line-no">381</span><span id="line-381"></span> |
| <span class="source-line-no">382</span><span id="line-382"> // It creates the ByteBuff and CallCleanup and assign to Connection instance.</span> |
| <span class="source-line-no">383</span><span id="line-383"> private void initByteBuffToReadInto(int length) {</span> |
| <span class="source-line-no">384</span><span id="line-384"> this.data = rpcServer.bbAllocator.allocate(length);</span> |
| <span class="source-line-no">385</span><span id="line-385"> this.callCleanup = data::release;</span> |
| <span class="source-line-no">386</span><span id="line-386"> }</span> |
| <span class="source-line-no">387</span><span id="line-387"></span> |
| <span class="source-line-no">388</span><span id="line-388"> protected int channelDataRead(ReadableByteChannel channel, ByteBuff buf) throws IOException {</span> |
| <span class="source-line-no">389</span><span id="line-389"> int count = buf.read(channel);</span> |
| <span class="source-line-no">390</span><span id="line-390"> if (count > 0) {</span> |
| <span class="source-line-no">391</span><span id="line-391"> this.rpcServer.metrics.receivedBytes(count);</span> |
| <span class="source-line-no">392</span><span id="line-392"> }</span> |
| <span class="source-line-no">393</span><span id="line-393"> return count;</span> |
| <span class="source-line-no">394</span><span id="line-394"> }</span> |
| <span class="source-line-no">395</span><span id="line-395"></span> |
| <span class="source-line-no">396</span><span id="line-396"> /**</span> |
| <span class="source-line-no">397</span><span id="line-397"> * Process the data buffer and clean the connection state for the next call.</span> |
| <span class="source-line-no">398</span><span id="line-398"> */</span> |
| <span class="source-line-no">399</span><span id="line-399"> private void process() throws IOException, InterruptedException {</span> |
| <span class="source-line-no">400</span><span id="line-400"> data.rewind();</span> |
| <span class="source-line-no">401</span><span id="line-401"> try {</span> |
| <span class="source-line-no">402</span><span id="line-402"> if (skipInitialSaslHandshake) {</span> |
| <span class="source-line-no">403</span><span id="line-403"> skipInitialSaslHandshake = false;</span> |
| <span class="source-line-no">404</span><span id="line-404"> return;</span> |
| <span class="source-line-no">405</span><span id="line-405"> }</span> |
| <span class="source-line-no">406</span><span id="line-406"></span> |
| <span class="source-line-no">407</span><span id="line-407"> if (useSasl) {</span> |
| <span class="source-line-no">408</span><span id="line-408"> saslReadAndProcess(data);</span> |
| <span class="source-line-no">409</span><span id="line-409"> } else {</span> |
| <span class="source-line-no">410</span><span id="line-410"> processOneRpc(data);</span> |
| <span class="source-line-no">411</span><span id="line-411"> }</span> |
| <span class="source-line-no">412</span><span id="line-412"> } catch (Exception e) {</span> |
| <span class="source-line-no">413</span><span id="line-413"> callCleanupIfNeeded();</span> |
| <span class="source-line-no">414</span><span id="line-414"> throw e;</span> |
| <span class="source-line-no">415</span><span id="line-415"> } finally {</span> |
| <span class="source-line-no">416</span><span id="line-416"> dataLengthBuffer.clear(); // Clean for the next call</span> |
| <span class="source-line-no">417</span><span id="line-417"> data = null; // For the GC</span> |
| <span class="source-line-no">418</span><span id="line-418"> this.callCleanup = null;</span> |
| <span class="source-line-no">419</span><span id="line-419"> }</span> |
| <span class="source-line-no">420</span><span id="line-420"> }</span> |
| <span class="source-line-no">421</span><span id="line-421"></span> |
| <span class="source-line-no">422</span><span id="line-422"> @Override</span> |
| <span class="source-line-no">423</span><span id="line-423"> public synchronized void close() {</span> |
| <span class="source-line-no">424</span><span id="line-424"> disposeSasl();</span> |
| <span class="source-line-no">425</span><span id="line-425"> data = null;</span> |
| <span class="source-line-no">426</span><span id="line-426"> callCleanupIfNeeded();</span> |
| <span class="source-line-no">427</span><span id="line-427"> if (!channel.isOpen()) {</span> |
| <span class="source-line-no">428</span><span id="line-428"> return;</span> |
| <span class="source-line-no">429</span><span id="line-429"> }</span> |
| <span class="source-line-no">430</span><span id="line-430"> try {</span> |
| <span class="source-line-no">431</span><span id="line-431"> socket.shutdownOutput();</span> |
| <span class="source-line-no">432</span><span id="line-432"> } catch (Exception ignored) {</span> |
| <span class="source-line-no">433</span><span id="line-433"> if (SimpleRpcServer.LOG.isTraceEnabled()) {</span> |
| <span class="source-line-no">434</span><span id="line-434"> SimpleRpcServer.LOG.trace("Ignored exception", ignored);</span> |
| <span class="source-line-no">435</span><span id="line-435"> }</span> |
| <span class="source-line-no">436</span><span id="line-436"> }</span> |
| <span class="source-line-no">437</span><span id="line-437"> if (channel.isOpen()) {</span> |
| <span class="source-line-no">438</span><span id="line-438"> try {</span> |
| <span class="source-line-no">439</span><span id="line-439"> channel.close();</span> |
| <span class="source-line-no">440</span><span id="line-440"> } catch (Exception ignored) {</span> |
| <span class="source-line-no">441</span><span id="line-441"> }</span> |
| <span class="source-line-no">442</span><span id="line-442"> }</span> |
| <span class="source-line-no">443</span><span id="line-443"> try {</span> |
| <span class="source-line-no">444</span><span id="line-444"> socket.close();</span> |
| <span class="source-line-no">445</span><span id="line-445"> } catch (Exception ignored) {</span> |
| <span class="source-line-no">446</span><span id="line-446"> if (SimpleRpcServer.LOG.isTraceEnabled()) {</span> |
| <span class="source-line-no">447</span><span id="line-447"> SimpleRpcServer.LOG.trace("Ignored exception", ignored);</span> |
| <span class="source-line-no">448</span><span id="line-448"> }</span> |
| <span class="source-line-no">449</span><span id="line-449"> }</span> |
| <span class="source-line-no">450</span><span id="line-450"> }</span> |
| <span class="source-line-no">451</span><span id="line-451"></span> |
| <span class="source-line-no">452</span><span id="line-452"> @Override</span> |
| <span class="source-line-no">453</span><span id="line-453"> public boolean isConnectionOpen() {</span> |
| <span class="source-line-no">454</span><span id="line-454"> return channel.isOpen();</span> |
| <span class="source-line-no">455</span><span id="line-455"> }</span> |
| <span class="source-line-no">456</span><span id="line-456"></span> |
| <span class="source-line-no">457</span><span id="line-457"> @Override</span> |
| <span class="source-line-no">458</span><span id="line-458"> public SimpleServerCall createCall(int id, BlockingService service, MethodDescriptor md,</span> |
| <span class="source-line-no">459</span><span id="line-459"> RequestHeader header, Message param, ExtendedCellScanner cellScanner, long size,</span> |
| <span class="source-line-no">460</span><span id="line-460"> InetAddress remoteAddress, int timeout, CallCleanup reqCleanup) {</span> |
| <span class="source-line-no">461</span><span id="line-461"> return new SimpleServerCall(id, service, md, header, param, cellScanner, this, size,</span> |
| <span class="source-line-no">462</span><span id="line-462"> remoteAddress, EnvironmentEdgeManager.currentTime(), timeout, this.rpcServer.bbAllocator,</span> |
| <span class="source-line-no">463</span><span id="line-463"> this.rpcServer.cellBlockBuilder, reqCleanup, this.responder);</span> |
| <span class="source-line-no">464</span><span id="line-464"> }</span> |
| <span class="source-line-no">465</span><span id="line-465"></span> |
| <span class="source-line-no">466</span><span id="line-466"> @Override</span> |
| <span class="source-line-no">467</span><span id="line-467"> protected void doRespond(RpcResponse resp) throws IOException {</span> |
| <span class="source-line-no">468</span><span id="line-468"> responder.doRespond(this, resp);</span> |
| <span class="source-line-no">469</span><span id="line-469"> }</span> |
| <span class="source-line-no">470</span><span id="line-470">}</span> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| </pre> |
| </div> |
| </main> |
| </body> |
| </html> |