| <!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: SimpleRpcServer, class: ConnectionManager"> |
| <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.IOException;</span> |
| <span class="source-line-no">021</span><span id="line-21">import java.net.BindException;</span> |
| <span class="source-line-no">022</span><span id="line-22">import java.net.InetSocketAddress;</span> |
| <span class="source-line-no">023</span><span id="line-23">import java.net.ServerSocket;</span> |
| <span class="source-line-no">024</span><span id="line-24">import java.net.SocketException;</span> |
| <span class="source-line-no">025</span><span id="line-25">import java.net.UnknownHostException;</span> |
| <span class="source-line-no">026</span><span id="line-26">import java.nio.channels.CancelledKeyException;</span> |
| <span class="source-line-no">027</span><span id="line-27">import java.nio.channels.GatheringByteChannel;</span> |
| <span class="source-line-no">028</span><span id="line-28">import java.nio.channels.SelectionKey;</span> |
| <span class="source-line-no">029</span><span id="line-29">import java.nio.channels.Selector;</span> |
| <span class="source-line-no">030</span><span id="line-30">import java.nio.channels.ServerSocketChannel;</span> |
| <span class="source-line-no">031</span><span id="line-31">import java.nio.channels.SocketChannel;</span> |
| <span class="source-line-no">032</span><span id="line-32">import java.util.Collections;</span> |
| <span class="source-line-no">033</span><span id="line-33">import java.util.Iterator;</span> |
| <span class="source-line-no">034</span><span id="line-34">import java.util.List;</span> |
| <span class="source-line-no">035</span><span id="line-35">import java.util.Set;</span> |
| <span class="source-line-no">036</span><span id="line-36">import java.util.Timer;</span> |
| <span class="source-line-no">037</span><span id="line-37">import java.util.TimerTask;</span> |
| <span class="source-line-no">038</span><span id="line-38">import java.util.concurrent.ConcurrentHashMap;</span> |
| <span class="source-line-no">039</span><span id="line-39">import java.util.concurrent.ExecutorService;</span> |
| <span class="source-line-no">040</span><span id="line-40">import java.util.concurrent.Executors;</span> |
| <span class="source-line-no">041</span><span id="line-41">import java.util.concurrent.LinkedBlockingQueue;</span> |
| <span class="source-line-no">042</span><span id="line-42">import java.util.concurrent.atomic.AtomicInteger;</span> |
| <span class="source-line-no">043</span><span id="line-43">import org.apache.hadoop.conf.Configuration;</span> |
| <span class="source-line-no">044</span><span id="line-44">import org.apache.hadoop.hbase.HBaseInterfaceAudience;</span> |
| <span class="source-line-no">045</span><span id="line-45">import org.apache.hadoop.hbase.HConstants;</span> |
| <span class="source-line-no">046</span><span id="line-46">import org.apache.hadoop.hbase.Server;</span> |
| <span class="source-line-no">047</span><span id="line-47">import org.apache.hadoop.hbase.security.HBasePolicyProvider;</span> |
| <span class="source-line-no">048</span><span id="line-48">import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;</span> |
| <span class="source-line-no">049</span><span id="line-49">import org.apache.hadoop.hbase.util.Threads;</span> |
| <span class="source-line-no">050</span><span id="line-50">import org.apache.hadoop.io.IOUtils;</span> |
| <span class="source-line-no">051</span><span id="line-51">import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;</span> |
| <span class="source-line-no">052</span><span id="line-52">import org.apache.yetus.audience.InterfaceAudience;</span> |
| <span class="source-line-no">053</span><span id="line-53"></span> |
| <span class="source-line-no">054</span><span id="line-54">import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;</span> |
| <span class="source-line-no">055</span><span id="line-55"></span> |
| <span class="source-line-no">056</span><span id="line-56">/**</span> |
| <span class="source-line-no">057</span><span id="line-57"> * The RPC server with native java NIO implementation deriving from Hadoop to host protobuf</span> |
| <span class="source-line-no">058</span><span id="line-58"> * described Services. It's the original one before HBASE-17262, and the default RPC server for now.</span> |
| <span class="source-line-no">059</span><span id="line-59"> * An RpcServer instance has a Listener that hosts the socket. Listener has fixed number of Readers</span> |
| <span class="source-line-no">060</span><span id="line-60"> * in an ExecutorPool, 10 by default. The Listener does an accept and then round robin a Reader is</span> |
| <span class="source-line-no">061</span><span id="line-61"> * chosen to do the read. The reader is registered on Selector. Read does total read off the channel</span> |
| <span class="source-line-no">062</span><span id="line-62"> * and the parse from which it makes a Call. The call is wrapped in a CallRunner and passed to the</span> |
| <span class="source-line-no">063</span><span id="line-63"> * scheduler to be run. Reader goes back to see if more to be done and loops till done.</span> |
| <span class="source-line-no">064</span><span id="line-64"> * <p></span> |
| <span class="source-line-no">065</span><span id="line-65"> * Scheduler can be variously implemented but default simple scheduler has handlers to which it has</span> |
| <span class="source-line-no">066</span><span id="line-66"> * given the queues into which calls (i.e. CallRunner instances) are inserted. Handlers run taking</span> |
| <span class="source-line-no">067</span><span id="line-67"> * from the queue. They run the CallRunner#run method on each item gotten from queue and keep taking</span> |
| <span class="source-line-no">068</span><span id="line-68"> * while the server is up. CallRunner#run executes the call. When done, asks the included Call to</span> |
| <span class="source-line-no">069</span><span id="line-69"> * put itself on new queue for Responder to pull from and return result to client.</span> |
| <span class="source-line-no">070</span><span id="line-70"> * @see BlockingRpcClient</span> |
| <span class="source-line-no">071</span><span id="line-71"> */</span> |
| <span class="source-line-no">072</span><span id="line-72">@Deprecated()</span> |
| <span class="source-line-no">073</span><span id="line-73">@InterfaceAudience.LimitedPrivate({ HBaseInterfaceAudience.CONFIG })</span> |
| <span class="source-line-no">074</span><span id="line-74">public class SimpleRpcServer extends RpcServer {</span> |
| <span class="source-line-no">075</span><span id="line-75"></span> |
| <span class="source-line-no">076</span><span id="line-76"> protected int port; // port we listen on</span> |
| <span class="source-line-no">077</span><span id="line-77"> protected InetSocketAddress address; // inet address we listen on</span> |
| <span class="source-line-no">078</span><span id="line-78"> private int readThreads; // number of read threads</span> |
| <span class="source-line-no">079</span><span id="line-79"></span> |
| <span class="source-line-no">080</span><span id="line-80"> protected int socketSendBufferSize;</span> |
| <span class="source-line-no">081</span><span id="line-81"> protected final long purgeTimeout; // in milliseconds</span> |
| <span class="source-line-no">082</span><span id="line-82"></span> |
| <span class="source-line-no">083</span><span id="line-83"> // maintains the set of client connections and handles idle timeouts</span> |
| <span class="source-line-no">084</span><span id="line-84"> private ConnectionManager connectionManager;</span> |
| <span class="source-line-no">085</span><span id="line-85"> private Listener listener = null;</span> |
| <span class="source-line-no">086</span><span id="line-86"> protected SimpleRpcServerResponder responder = null;</span> |
| <span class="source-line-no">087</span><span id="line-87"></span> |
| <span class="source-line-no">088</span><span id="line-88"> /** Listens on the socket. Creates jobs for the handler threads */</span> |
| <span class="source-line-no">089</span><span id="line-89"> private class Listener extends Thread {</span> |
| <span class="source-line-no">090</span><span id="line-90"></span> |
| <span class="source-line-no">091</span><span id="line-91"> private ServerSocketChannel acceptChannel = null; // the accept channel</span> |
| <span class="source-line-no">092</span><span id="line-92"> private Selector selector = null; // the selector that we use for the server</span> |
| <span class="source-line-no">093</span><span id="line-93"> private Reader[] readers = null;</span> |
| <span class="source-line-no">094</span><span id="line-94"> private int currentReader = 0;</span> |
| <span class="source-line-no">095</span><span id="line-95"> private final int readerPendingConnectionQueueLength;</span> |
| <span class="source-line-no">096</span><span id="line-96"></span> |
| <span class="source-line-no">097</span><span id="line-97"> private ExecutorService readPool;</span> |
| <span class="source-line-no">098</span><span id="line-98"></span> |
| <span class="source-line-no">099</span><span id="line-99"> public Listener(final String name) throws IOException {</span> |
| <span class="source-line-no">100</span><span id="line-100"> super(name);</span> |
| <span class="source-line-no">101</span><span id="line-101"> // The backlog of requests that we will have the serversocket carry.</span> |
| <span class="source-line-no">102</span><span id="line-102"> int backlogLength = conf.getInt("hbase.ipc.server.listen.queue.size", 128);</span> |
| <span class="source-line-no">103</span><span id="line-103"> readerPendingConnectionQueueLength =</span> |
| <span class="source-line-no">104</span><span id="line-104"> conf.getInt("hbase.ipc.server.read.connection-queue.size", 100);</span> |
| <span class="source-line-no">105</span><span id="line-105"> // Create a new server socket and set to non blocking mode</span> |
| <span class="source-line-no">106</span><span id="line-106"> acceptChannel = ServerSocketChannel.open();</span> |
| <span class="source-line-no">107</span><span id="line-107"> acceptChannel.configureBlocking(false);</span> |
| <span class="source-line-no">108</span><span id="line-108"></span> |
| <span class="source-line-no">109</span><span id="line-109"> // Bind the server socket to the binding addrees (can be different from the default interface)</span> |
| <span class="source-line-no">110</span><span id="line-110"> bind(acceptChannel.socket(), bindAddress, backlogLength);</span> |
| <span class="source-line-no">111</span><span id="line-111"> port = acceptChannel.socket().getLocalPort(); // Could be an ephemeral port</span> |
| <span class="source-line-no">112</span><span id="line-112"> address = (InetSocketAddress) acceptChannel.socket().getLocalSocketAddress();</span> |
| <span class="source-line-no">113</span><span id="line-113"> // create a selector;</span> |
| <span class="source-line-no">114</span><span id="line-114"> selector = Selector.open();</span> |
| <span class="source-line-no">115</span><span id="line-115"></span> |
| <span class="source-line-no">116</span><span id="line-116"> readers = new Reader[readThreads];</span> |
| <span class="source-line-no">117</span><span id="line-117"> // Why this executor thing? Why not like hadoop just start up all the threads? I suppose it</span> |
| <span class="source-line-no">118</span><span id="line-118"> // has an advantage in that it is easy to shutdown the pool.</span> |
| <span class="source-line-no">119</span><span id="line-119"> readPool = Executors.newFixedThreadPool(readThreads,</span> |
| <span class="source-line-no">120</span><span id="line-120"> new ThreadFactoryBuilder()</span> |
| <span class="source-line-no">121</span><span id="line-121"> .setNameFormat("Reader=%d,bindAddress=" + bindAddress.getHostName() + ",port=" + port)</span> |
| <span class="source-line-no">122</span><span id="line-122"> .setDaemon(true).setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build());</span> |
| <span class="source-line-no">123</span><span id="line-123"> for (int i = 0; i < readThreads; ++i) {</span> |
| <span class="source-line-no">124</span><span id="line-124"> Reader reader = new Reader();</span> |
| <span class="source-line-no">125</span><span id="line-125"> readers[i] = reader;</span> |
| <span class="source-line-no">126</span><span id="line-126"> readPool.execute(reader);</span> |
| <span class="source-line-no">127</span><span id="line-127"> }</span> |
| <span class="source-line-no">128</span><span id="line-128"> LOG.info(getName() + ": started " + readThreads + " reader(s) listening on port=" + port);</span> |
| <span class="source-line-no">129</span><span id="line-129"></span> |
| <span class="source-line-no">130</span><span id="line-130"> // Register accepts on the server socket with the selector.</span> |
| <span class="source-line-no">131</span><span id="line-131"> acceptChannel.register(selector, SelectionKey.OP_ACCEPT);</span> |
| <span class="source-line-no">132</span><span id="line-132"> this.setName("Listener,port=" + port);</span> |
| <span class="source-line-no">133</span><span id="line-133"> this.setDaemon(true);</span> |
| <span class="source-line-no">134</span><span id="line-134"> }</span> |
| <span class="source-line-no">135</span><span id="line-135"></span> |
| <span class="source-line-no">136</span><span id="line-136"> private class Reader implements Runnable {</span> |
| <span class="source-line-no">137</span><span id="line-137"> final private LinkedBlockingQueue<SimpleServerRpcConnection> pendingConnections;</span> |
| <span class="source-line-no">138</span><span id="line-138"> private final Selector readSelector;</span> |
| <span class="source-line-no">139</span><span id="line-139"></span> |
| <span class="source-line-no">140</span><span id="line-140"> Reader() throws IOException {</span> |
| <span class="source-line-no">141</span><span id="line-141"> this.pendingConnections = new LinkedBlockingQueue<>(readerPendingConnectionQueueLength);</span> |
| <span class="source-line-no">142</span><span id="line-142"> this.readSelector = Selector.open();</span> |
| <span class="source-line-no">143</span><span id="line-143"> }</span> |
| <span class="source-line-no">144</span><span id="line-144"></span> |
| <span class="source-line-no">145</span><span id="line-145"> @Override</span> |
| <span class="source-line-no">146</span><span id="line-146"> public void run() {</span> |
| <span class="source-line-no">147</span><span id="line-147"> try {</span> |
| <span class="source-line-no">148</span><span id="line-148"> doRunLoop();</span> |
| <span class="source-line-no">149</span><span id="line-149"> } finally {</span> |
| <span class="source-line-no">150</span><span id="line-150"> try {</span> |
| <span class="source-line-no">151</span><span id="line-151"> readSelector.close();</span> |
| <span class="source-line-no">152</span><span id="line-152"> } catch (IOException ioe) {</span> |
| <span class="source-line-no">153</span><span id="line-153"> LOG.error(getName() + ": error closing read selector in " + getName(), ioe);</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"></span> |
| <span class="source-line-no">158</span><span id="line-158"> private synchronized void doRunLoop() {</span> |
| <span class="source-line-no">159</span><span id="line-159"> while (running) {</span> |
| <span class="source-line-no">160</span><span id="line-160"> try {</span> |
| <span class="source-line-no">161</span><span id="line-161"> // Consume as many connections as currently queued to avoid</span> |
| <span class="source-line-no">162</span><span id="line-162"> // unbridled acceptance of connections that starves the select</span> |
| <span class="source-line-no">163</span><span id="line-163"> int size = pendingConnections.size();</span> |
| <span class="source-line-no">164</span><span id="line-164"> for (int i = size; i > 0; i--) {</span> |
| <span class="source-line-no">165</span><span id="line-165"> SimpleServerRpcConnection conn = pendingConnections.take();</span> |
| <span class="source-line-no">166</span><span id="line-166"> conn.channel.register(readSelector, SelectionKey.OP_READ, conn);</span> |
| <span class="source-line-no">167</span><span id="line-167"> }</span> |
| <span class="source-line-no">168</span><span id="line-168"> readSelector.select();</span> |
| <span class="source-line-no">169</span><span id="line-169"> Iterator<SelectionKey> iter = readSelector.selectedKeys().iterator();</span> |
| <span class="source-line-no">170</span><span id="line-170"> while (iter.hasNext()) {</span> |
| <span class="source-line-no">171</span><span id="line-171"> SelectionKey key = iter.next();</span> |
| <span class="source-line-no">172</span><span id="line-172"> iter.remove();</span> |
| <span class="source-line-no">173</span><span id="line-173"> if (key.isValid()) {</span> |
| <span class="source-line-no">174</span><span id="line-174"> if (key.isReadable()) {</span> |
| <span class="source-line-no">175</span><span id="line-175"> doRead(key);</span> |
| <span class="source-line-no">176</span><span id="line-176"> }</span> |
| <span class="source-line-no">177</span><span id="line-177"> }</span> |
| <span class="source-line-no">178</span><span id="line-178"> key = null;</span> |
| <span class="source-line-no">179</span><span id="line-179"> }</span> |
| <span class="source-line-no">180</span><span id="line-180"> } catch (InterruptedException e) {</span> |
| <span class="source-line-no">181</span><span id="line-181"> if (running) { // unexpected -- log it</span> |
| <span class="source-line-no">182</span><span id="line-182"> LOG.info(Thread.currentThread().getName() + " unexpectedly interrupted", e);</span> |
| <span class="source-line-no">183</span><span id="line-183"> }</span> |
| <span class="source-line-no">184</span><span id="line-184"> } catch (CancelledKeyException e) {</span> |
| <span class="source-line-no">185</span><span id="line-185"> LOG.error(getName() + ": CancelledKeyException in Reader", e);</span> |
| <span class="source-line-no">186</span><span id="line-186"> } catch (IOException ex) {</span> |
| <span class="source-line-no">187</span><span id="line-187"> LOG.info(getName() + ": IOException in Reader", ex);</span> |
| <span class="source-line-no">188</span><span id="line-188"> }</span> |
| <span class="source-line-no">189</span><span id="line-189"> }</span> |
| <span class="source-line-no">190</span><span id="line-190"> }</span> |
| <span class="source-line-no">191</span><span id="line-191"></span> |
| <span class="source-line-no">192</span><span id="line-192"> /**</span> |
| <span class="source-line-no">193</span><span id="line-193"> * Updating the readSelector while it's being used is not thread-safe, so the connection must</span> |
| <span class="source-line-no">194</span><span id="line-194"> * be queued. The reader will drain the queue and update its readSelector before performing</span> |
| <span class="source-line-no">195</span><span id="line-195"> * the next select</span> |
| <span class="source-line-no">196</span><span id="line-196"> */</span> |
| <span class="source-line-no">197</span><span id="line-197"> public void addConnection(SimpleServerRpcConnection conn) throws IOException {</span> |
| <span class="source-line-no">198</span><span id="line-198"> pendingConnections.add(conn);</span> |
| <span class="source-line-no">199</span><span id="line-199"> readSelector.wakeup();</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"> @Override</span> |
| <span class="source-line-no">204</span><span id="line-204"> @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC",</span> |
| <span class="source-line-no">205</span><span id="line-205"> justification = "selector access is not synchronized; seems fine but concerned changing "</span> |
| <span class="source-line-no">206</span><span id="line-206"> + "it will have per impact")</span> |
| <span class="source-line-no">207</span><span id="line-207"> public void run() {</span> |
| <span class="source-line-no">208</span><span id="line-208"> LOG.info(getName() + ": starting");</span> |
| <span class="source-line-no">209</span><span id="line-209"> connectionManager.startIdleScan();</span> |
| <span class="source-line-no">210</span><span id="line-210"> while (running) {</span> |
| <span class="source-line-no">211</span><span id="line-211"> SelectionKey key = null;</span> |
| <span class="source-line-no">212</span><span id="line-212"> try {</span> |
| <span class="source-line-no">213</span><span id="line-213"> selector.select(); // FindBugs IS2_INCONSISTENT_SYNC</span> |
| <span class="source-line-no">214</span><span id="line-214"> Iterator<SelectionKey> iter = selector.selectedKeys().iterator();</span> |
| <span class="source-line-no">215</span><span id="line-215"> while (iter.hasNext()) {</span> |
| <span class="source-line-no">216</span><span id="line-216"> key = iter.next();</span> |
| <span class="source-line-no">217</span><span id="line-217"> iter.remove();</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"> if (key.isValid()) {</span> |
| <span class="source-line-no">220</span><span id="line-220"> if (key.isAcceptable()) doAccept(key);</span> |
| <span class="source-line-no">221</span><span id="line-221"> }</span> |
| <span class="source-line-no">222</span><span id="line-222"> } catch (IOException ignored) {</span> |
| <span class="source-line-no">223</span><span id="line-223"> if (LOG.isTraceEnabled()) LOG.trace("ignored", ignored);</span> |
| <span class="source-line-no">224</span><span id="line-224"> }</span> |
| <span class="source-line-no">225</span><span id="line-225"> key = null;</span> |
| <span class="source-line-no">226</span><span id="line-226"> }</span> |
| <span class="source-line-no">227</span><span id="line-227"> } catch (OutOfMemoryError e) {</span> |
| <span class="source-line-no">228</span><span id="line-228"> if (errorHandler != null) {</span> |
| <span class="source-line-no">229</span><span id="line-229"> if (errorHandler.checkOOME(e)) {</span> |
| <span class="source-line-no">230</span><span id="line-230"> LOG.info(getName() + ": exiting on OutOfMemoryError");</span> |
| <span class="source-line-no">231</span><span id="line-231"> closeCurrentConnection(key, e);</span> |
| <span class="source-line-no">232</span><span id="line-232"> connectionManager.closeIdle(true);</span> |
| <span class="source-line-no">233</span><span id="line-233"> return;</span> |
| <span class="source-line-no">234</span><span id="line-234"> }</span> |
| <span class="source-line-no">235</span><span id="line-235"> } else {</span> |
| <span class="source-line-no">236</span><span id="line-236"> // we can run out of memory if we have too many threads</span> |
| <span class="source-line-no">237</span><span id="line-237"> // log the event and sleep for a minute and give</span> |
| <span class="source-line-no">238</span><span id="line-238"> // some thread(s) a chance to finish</span> |
| <span class="source-line-no">239</span><span id="line-239"> LOG.warn(getName() + ": OutOfMemoryError in server select", e);</span> |
| <span class="source-line-no">240</span><span id="line-240"> closeCurrentConnection(key, e);</span> |
| <span class="source-line-no">241</span><span id="line-241"> connectionManager.closeIdle(true);</span> |
| <span class="source-line-no">242</span><span id="line-242"> try {</span> |
| <span class="source-line-no">243</span><span id="line-243"> Thread.sleep(60000);</span> |
| <span class="source-line-no">244</span><span id="line-244"> } catch (InterruptedException ex) {</span> |
| <span class="source-line-no">245</span><span id="line-245"> LOG.debug("Interrupted while sleeping");</span> |
| <span class="source-line-no">246</span><span id="line-246"> }</span> |
| <span class="source-line-no">247</span><span id="line-247"> }</span> |
| <span class="source-line-no">248</span><span id="line-248"> } catch (Exception e) {</span> |
| <span class="source-line-no">249</span><span id="line-249"> closeCurrentConnection(key, e);</span> |
| <span class="source-line-no">250</span><span id="line-250"> }</span> |
| <span class="source-line-no">251</span><span id="line-251"> }</span> |
| <span class="source-line-no">252</span><span id="line-252"> LOG.info(getName() + ": stopping");</span> |
| <span class="source-line-no">253</span><span id="line-253"> synchronized (this) {</span> |
| <span class="source-line-no">254</span><span id="line-254"> try {</span> |
| <span class="source-line-no">255</span><span id="line-255"> acceptChannel.close();</span> |
| <span class="source-line-no">256</span><span id="line-256"> selector.close();</span> |
| <span class="source-line-no">257</span><span id="line-257"> } catch (IOException ignored) {</span> |
| <span class="source-line-no">258</span><span id="line-258"> if (LOG.isTraceEnabled()) LOG.trace("ignored", ignored);</span> |
| <span class="source-line-no">259</span><span id="line-259"> }</span> |
| <span class="source-line-no">260</span><span id="line-260"></span> |
| <span class="source-line-no">261</span><span id="line-261"> selector = null;</span> |
| <span class="source-line-no">262</span><span id="line-262"> acceptChannel = null;</span> |
| <span class="source-line-no">263</span><span id="line-263"></span> |
| <span class="source-line-no">264</span><span id="line-264"> // close all connections</span> |
| <span class="source-line-no">265</span><span id="line-265"> connectionManager.stopIdleScan();</span> |
| <span class="source-line-no">266</span><span id="line-266"> connectionManager.closeAll();</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"> private void closeCurrentConnection(SelectionKey key, Throwable e) {</span> |
| <span class="source-line-no">271</span><span id="line-271"> if (key != null) {</span> |
| <span class="source-line-no">272</span><span id="line-272"> SimpleServerRpcConnection c = (SimpleServerRpcConnection) key.attachment();</span> |
| <span class="source-line-no">273</span><span id="line-273"> if (c != null) {</span> |
| <span class="source-line-no">274</span><span id="line-274"> closeConnection(c);</span> |
| <span class="source-line-no">275</span><span id="line-275"> key.attach(null);</span> |
| <span class="source-line-no">276</span><span id="line-276"> }</span> |
| <span class="source-line-no">277</span><span id="line-277"> }</span> |
| <span class="source-line-no">278</span><span id="line-278"> }</span> |
| <span class="source-line-no">279</span><span id="line-279"></span> |
| <span class="source-line-no">280</span><span id="line-280"> InetSocketAddress getAddress() {</span> |
| <span class="source-line-no">281</span><span id="line-281"> return address;</span> |
| <span class="source-line-no">282</span><span id="line-282"> }</span> |
| <span class="source-line-no">283</span><span id="line-283"></span> |
| <span class="source-line-no">284</span><span id="line-284"> void doAccept(SelectionKey key) throws InterruptedException, IOException, OutOfMemoryError {</span> |
| <span class="source-line-no">285</span><span id="line-285"> ServerSocketChannel server = (ServerSocketChannel) key.channel();</span> |
| <span class="source-line-no">286</span><span id="line-286"> SocketChannel channel;</span> |
| <span class="source-line-no">287</span><span id="line-287"> while ((channel = server.accept()) != null) {</span> |
| <span class="source-line-no">288</span><span id="line-288"> channel.configureBlocking(false);</span> |
| <span class="source-line-no">289</span><span id="line-289"> channel.socket().setTcpNoDelay(tcpNoDelay);</span> |
| <span class="source-line-no">290</span><span id="line-290"> channel.socket().setKeepAlive(tcpKeepAlive);</span> |
| <span class="source-line-no">291</span><span id="line-291"> Reader reader = getReader();</span> |
| <span class="source-line-no">292</span><span id="line-292"> SimpleServerRpcConnection c = connectionManager.register(channel);</span> |
| <span class="source-line-no">293</span><span id="line-293"> // If the connectionManager can't take it, close the connection.</span> |
| <span class="source-line-no">294</span><span id="line-294"> if (c == null) {</span> |
| <span class="source-line-no">295</span><span id="line-295"> if (channel.isOpen()) {</span> |
| <span class="source-line-no">296</span><span id="line-296"> IOUtils.cleanupWithLogger(LOG, channel);</span> |
| <span class="source-line-no">297</span><span id="line-297"> }</span> |
| <span class="source-line-no">298</span><span id="line-298"> continue;</span> |
| <span class="source-line-no">299</span><span id="line-299"> }</span> |
| <span class="source-line-no">300</span><span id="line-300"> key.attach(c); // so closeCurrentConnection can get the object</span> |
| <span class="source-line-no">301</span><span id="line-301"> reader.addConnection(c);</span> |
| <span class="source-line-no">302</span><span id="line-302"> }</span> |
| <span class="source-line-no">303</span><span id="line-303"> }</span> |
| <span class="source-line-no">304</span><span id="line-304"></span> |
| <span class="source-line-no">305</span><span id="line-305"> void doRead(SelectionKey key) throws InterruptedException {</span> |
| <span class="source-line-no">306</span><span id="line-306"> int count;</span> |
| <span class="source-line-no">307</span><span id="line-307"> SimpleServerRpcConnection c = (SimpleServerRpcConnection) key.attachment();</span> |
| <span class="source-line-no">308</span><span id="line-308"> if (c == null) {</span> |
| <span class="source-line-no">309</span><span id="line-309"> return;</span> |
| <span class="source-line-no">310</span><span id="line-310"> }</span> |
| <span class="source-line-no">311</span><span id="line-311"> c.setLastContact(EnvironmentEdgeManager.currentTime());</span> |
| <span class="source-line-no">312</span><span id="line-312"> try {</span> |
| <span class="source-line-no">313</span><span id="line-313"> count = c.readAndProcess();</span> |
| <span class="source-line-no">314</span><span id="line-314"> } catch (InterruptedException ieo) {</span> |
| <span class="source-line-no">315</span><span id="line-315"> LOG.info(Thread.currentThread().getName() + ": readAndProcess caught InterruptedException",</span> |
| <span class="source-line-no">316</span><span id="line-316"> ieo);</span> |
| <span class="source-line-no">317</span><span id="line-317"> throw ieo;</span> |
| <span class="source-line-no">318</span><span id="line-318"> } catch (Exception e) {</span> |
| <span class="source-line-no">319</span><span id="line-319"> if (LOG.isDebugEnabled()) {</span> |
| <span class="source-line-no">320</span><span id="line-320"> LOG.debug("Caught exception while reading:", e);</span> |
| <span class="source-line-no">321</span><span id="line-321"> }</span> |
| <span class="source-line-no">322</span><span id="line-322"> count = -1; // so that the (count < 0) block is executed</span> |
| <span class="source-line-no">323</span><span id="line-323"> }</span> |
| <span class="source-line-no">324</span><span id="line-324"> if (count < 0) {</span> |
| <span class="source-line-no">325</span><span id="line-325"> closeConnection(c);</span> |
| <span class="source-line-no">326</span><span id="line-326"> c = null;</span> |
| <span class="source-line-no">327</span><span id="line-327"> } else {</span> |
| <span class="source-line-no">328</span><span id="line-328"> c.setLastContact(EnvironmentEdgeManager.currentTime());</span> |
| <span class="source-line-no">329</span><span id="line-329"> }</span> |
| <span class="source-line-no">330</span><span id="line-330"> }</span> |
| <span class="source-line-no">331</span><span id="line-331"></span> |
| <span class="source-line-no">332</span><span id="line-332"> synchronized void doStop() {</span> |
| <span class="source-line-no">333</span><span id="line-333"> if (selector != null) {</span> |
| <span class="source-line-no">334</span><span id="line-334"> selector.wakeup();</span> |
| <span class="source-line-no">335</span><span id="line-335"> Thread.yield();</span> |
| <span class="source-line-no">336</span><span id="line-336"> }</span> |
| <span class="source-line-no">337</span><span id="line-337"> if (acceptChannel != null) {</span> |
| <span class="source-line-no">338</span><span id="line-338"> try {</span> |
| <span class="source-line-no">339</span><span id="line-339"> acceptChannel.socket().close();</span> |
| <span class="source-line-no">340</span><span id="line-340"> } catch (IOException e) {</span> |
| <span class="source-line-no">341</span><span id="line-341"> LOG.info(getName() + ": exception in closing listener socket. " + e);</span> |
| <span class="source-line-no">342</span><span id="line-342"> }</span> |
| <span class="source-line-no">343</span><span id="line-343"> }</span> |
| <span class="source-line-no">344</span><span id="line-344"> readPool.shutdownNow();</span> |
| <span class="source-line-no">345</span><span id="line-345"> }</span> |
| <span class="source-line-no">346</span><span id="line-346"></span> |
| <span class="source-line-no">347</span><span id="line-347"> // The method that will return the next reader to work with</span> |
| <span class="source-line-no">348</span><span id="line-348"> // Simplistic implementation of round robin for now</span> |
| <span class="source-line-no">349</span><span id="line-349"> Reader getReader() {</span> |
| <span class="source-line-no">350</span><span id="line-350"> currentReader = (currentReader + 1) % readers.length;</span> |
| <span class="source-line-no">351</span><span id="line-351"> return readers[currentReader];</span> |
| <span class="source-line-no">352</span><span id="line-352"> }</span> |
| <span class="source-line-no">353</span><span id="line-353"> }</span> |
| <span class="source-line-no">354</span><span id="line-354"></span> |
| <span class="source-line-no">355</span><span id="line-355"> /**</span> |
| <span class="source-line-no">356</span><span id="line-356"> * Constructs a server listening on the named port and address.</span> |
| <span class="source-line-no">357</span><span id="line-357"> * @param server hosting instance of {@link Server}. We will do authentications if an</span> |
| <span class="source-line-no">358</span><span id="line-358"> * instance else pass null for no authentication check.</span> |
| <span class="source-line-no">359</span><span id="line-359"> * @param name Used keying this rpc servers' metrics and for naming the Listener</span> |
| <span class="source-line-no">360</span><span id="line-360"> * thread.</span> |
| <span class="source-line-no">361</span><span id="line-361"> * @param services A list of services.</span> |
| <span class="source-line-no">362</span><span id="line-362"> * @param bindAddress Where to listen</span> |
| <span class="source-line-no">363</span><span id="line-363"> * @param reservoirEnabled Enable ByteBufferPool or not.</span> |
| <span class="source-line-no">364</span><span id="line-364"> */</span> |
| <span class="source-line-no">365</span><span id="line-365"> public SimpleRpcServer(final Server server, final String name,</span> |
| <span class="source-line-no">366</span><span id="line-366"> final List<BlockingServiceAndInterface> services, final InetSocketAddress bindAddress,</span> |
| <span class="source-line-no">367</span><span id="line-367"> Configuration conf, RpcScheduler scheduler, boolean reservoirEnabled) throws IOException {</span> |
| <span class="source-line-no">368</span><span id="line-368"> super(server, name, services, bindAddress, conf, scheduler, reservoirEnabled);</span> |
| <span class="source-line-no">369</span><span id="line-369"> this.socketSendBufferSize = 0;</span> |
| <span class="source-line-no">370</span><span id="line-370"> this.readThreads = conf.getInt("hbase.ipc.server.read.threadpool.size", 10);</span> |
| <span class="source-line-no">371</span><span id="line-371"> this.purgeTimeout =</span> |
| <span class="source-line-no">372</span><span id="line-372"> conf.getLong("hbase.ipc.client.call.purge.timeout", 2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT);</span> |
| <span class="source-line-no">373</span><span id="line-373"></span> |
| <span class="source-line-no">374</span><span id="line-374"> // Start the listener here and let it bind to the port</span> |
| <span class="source-line-no">375</span><span id="line-375"> listener = new Listener(name);</span> |
| <span class="source-line-no">376</span><span id="line-376"> this.port = listener.getAddress().getPort();</span> |
| <span class="source-line-no">377</span><span id="line-377"></span> |
| <span class="source-line-no">378</span><span id="line-378"> // Create the responder here</span> |
| <span class="source-line-no">379</span><span id="line-379"> responder = new SimpleRpcServerResponder(this);</span> |
| <span class="source-line-no">380</span><span id="line-380"> connectionManager = new ConnectionManager();</span> |
| <span class="source-line-no">381</span><span id="line-381"> initReconfigurable(conf);</span> |
| <span class="source-line-no">382</span><span id="line-382"></span> |
| <span class="source-line-no">383</span><span id="line-383"> this.scheduler.init(new RpcSchedulerContext(this));</span> |
| <span class="source-line-no">384</span><span id="line-384"> }</span> |
| <span class="source-line-no">385</span><span id="line-385"></span> |
| <span class="source-line-no">386</span><span id="line-386"> /**</span> |
| <span class="source-line-no">387</span><span id="line-387"> * Subclasses of HBaseServer can override this to provide their own Connection implementations.</span> |
| <span class="source-line-no">388</span><span id="line-388"> */</span> |
| <span class="source-line-no">389</span><span id="line-389"> protected SimpleServerRpcConnection getConnection(SocketChannel channel, long time) {</span> |
| <span class="source-line-no">390</span><span id="line-390"> return new SimpleServerRpcConnection(this, channel, time);</span> |
| <span class="source-line-no">391</span><span id="line-391"> }</span> |
| <span class="source-line-no">392</span><span id="line-392"></span> |
| <span class="source-line-no">393</span><span id="line-393"> protected void closeConnection(SimpleServerRpcConnection connection) {</span> |
| <span class="source-line-no">394</span><span id="line-394"> connectionManager.close(connection);</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"> /**</span> |
| <span class="source-line-no">398</span><span id="line-398"> * Sets the socket buffer size used for responding to RPCs.</span> |
| <span class="source-line-no">399</span><span id="line-399"> * @param size send size</span> |
| <span class="source-line-no">400</span><span id="line-400"> */</span> |
| <span class="source-line-no">401</span><span id="line-401"> @Override</span> |
| <span class="source-line-no">402</span><span id="line-402"> public void setSocketSendBufSize(int size) {</span> |
| <span class="source-line-no">403</span><span id="line-403"> this.socketSendBufferSize = size;</span> |
| <span class="source-line-no">404</span><span id="line-404"> }</span> |
| <span class="source-line-no">405</span><span id="line-405"></span> |
| <span class="source-line-no">406</span><span id="line-406"> /** Starts the service. Must be called before any calls will be handled. */</span> |
| <span class="source-line-no">407</span><span id="line-407"> @Override</span> |
| <span class="source-line-no">408</span><span id="line-408"> public synchronized void start() {</span> |
| <span class="source-line-no">409</span><span id="line-409"> if (started) {</span> |
| <span class="source-line-no">410</span><span id="line-410"> return;</span> |
| <span class="source-line-no">411</span><span id="line-411"> }</span> |
| <span class="source-line-no">412</span><span id="line-412"> authTokenSecretMgr = createSecretManager();</span> |
| <span class="source-line-no">413</span><span id="line-413"> if (authTokenSecretMgr != null) {</span> |
| <span class="source-line-no">414</span><span id="line-414"> // Start AuthenticationTokenSecretManager in synchronized way to avoid race conditions in</span> |
| <span class="source-line-no">415</span><span id="line-415"> // LeaderElector start. See HBASE-25875</span> |
| <span class="source-line-no">416</span><span id="line-416"> synchronized (authTokenSecretMgr) {</span> |
| <span class="source-line-no">417</span><span id="line-417"> setSecretManager(authTokenSecretMgr);</span> |
| <span class="source-line-no">418</span><span id="line-418"> authTokenSecretMgr.start();</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"> this.authManager = new ServiceAuthorizationManager();</span> |
| <span class="source-line-no">422</span><span id="line-422"> HBasePolicyProvider.init(conf, authManager);</span> |
| <span class="source-line-no">423</span><span id="line-423"> responder.start();</span> |
| <span class="source-line-no">424</span><span id="line-424"> listener.start();</span> |
| <span class="source-line-no">425</span><span id="line-425"> scheduler.start();</span> |
| <span class="source-line-no">426</span><span id="line-426"> started = true;</span> |
| <span class="source-line-no">427</span><span id="line-427"> }</span> |
| <span class="source-line-no">428</span><span id="line-428"></span> |
| <span class="source-line-no">429</span><span id="line-429"> /** Stops the service. No new calls will be handled after this is called. */</span> |
| <span class="source-line-no">430</span><span id="line-430"> @Override</span> |
| <span class="source-line-no">431</span><span id="line-431"> public synchronized void stop() {</span> |
| <span class="source-line-no">432</span><span id="line-432"> LOG.info("Stopping server on " + port);</span> |
| <span class="source-line-no">433</span><span id="line-433"> running = false;</span> |
| <span class="source-line-no">434</span><span id="line-434"> if (authTokenSecretMgr != null) {</span> |
| <span class="source-line-no">435</span><span id="line-435"> authTokenSecretMgr.stop();</span> |
| <span class="source-line-no">436</span><span id="line-436"> authTokenSecretMgr = null;</span> |
| <span class="source-line-no">437</span><span id="line-437"> }</span> |
| <span class="source-line-no">438</span><span id="line-438"> listener.interrupt();</span> |
| <span class="source-line-no">439</span><span id="line-439"> listener.doStop();</span> |
| <span class="source-line-no">440</span><span id="line-440"> responder.interrupt();</span> |
| <span class="source-line-no">441</span><span id="line-441"> scheduler.stop();</span> |
| <span class="source-line-no">442</span><span id="line-442"> notifyAll();</span> |
| <span class="source-line-no">443</span><span id="line-443"> }</span> |
| <span class="source-line-no">444</span><span id="line-444"></span> |
| <span class="source-line-no">445</span><span id="line-445"> /**</span> |
| <span class="source-line-no">446</span><span id="line-446"> * Wait for the server to be stopped. Does not wait for all subthreads to finish.</span> |
| <span class="source-line-no">447</span><span id="line-447"> * @see #stop()</span> |
| <span class="source-line-no">448</span><span id="line-448"> */</span> |
| <span class="source-line-no">449</span><span id="line-449"> @Override</span> |
| <span class="source-line-no">450</span><span id="line-450"> public synchronized void join() throws InterruptedException {</span> |
| <span class="source-line-no">451</span><span id="line-451"> while (running) {</span> |
| <span class="source-line-no">452</span><span id="line-452"> wait();</span> |
| <span class="source-line-no">453</span><span id="line-453"> }</span> |
| <span class="source-line-no">454</span><span id="line-454"> }</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"> * Return the socket (ip+port) on which the RPC server is listening to. May return null if the</span> |
| <span class="source-line-no">458</span><span id="line-458"> * listener channel is closed.</span> |
| <span class="source-line-no">459</span><span id="line-459"> * @return the socket (ip+port) on which the RPC server is listening to, or null if this</span> |
| <span class="source-line-no">460</span><span id="line-460"> * information cannot be determined</span> |
| <span class="source-line-no">461</span><span id="line-461"> */</span> |
| <span class="source-line-no">462</span><span id="line-462"> @Override</span> |
| <span class="source-line-no">463</span><span id="line-463"> public synchronized InetSocketAddress getListenerAddress() {</span> |
| <span class="source-line-no">464</span><span id="line-464"> if (listener == null) {</span> |
| <span class="source-line-no">465</span><span id="line-465"> return null;</span> |
| <span class="source-line-no">466</span><span id="line-466"> }</span> |
| <span class="source-line-no">467</span><span id="line-467"> return listener.getAddress();</span> |
| <span class="source-line-no">468</span><span id="line-468"> }</span> |
| <span class="source-line-no">469</span><span id="line-469"></span> |
| <span class="source-line-no">470</span><span id="line-470"> protected long channelWrite(GatheringByteChannel channel, BufferChain bufferChain)</span> |
| <span class="source-line-no">471</span><span id="line-471"> throws IOException {</span> |
| <span class="source-line-no">472</span><span id="line-472"> long count = bufferChain.write(channel);</span> |
| <span class="source-line-no">473</span><span id="line-473"> if (count > 0) {</span> |
| <span class="source-line-no">474</span><span id="line-474"> this.metrics.sentBytes(count);</span> |
| <span class="source-line-no">475</span><span id="line-475"> }</span> |
| <span class="source-line-no">476</span><span id="line-476"> return count;</span> |
| <span class="source-line-no">477</span><span id="line-477"> }</span> |
| <span class="source-line-no">478</span><span id="line-478"></span> |
| <span class="source-line-no">479</span><span id="line-479"> /**</span> |
| <span class="source-line-no">480</span><span id="line-480"> * A convenience method to bind to a given address and report better exceptions if the address is</span> |
| <span class="source-line-no">481</span><span id="line-481"> * not a valid host.</span> |
| <span class="source-line-no">482</span><span id="line-482"> * @param socket the socket to bind</span> |
| <span class="source-line-no">483</span><span id="line-483"> * @param address the address to bind to</span> |
| <span class="source-line-no">484</span><span id="line-484"> * @param backlog the number of connections allowed in the queue</span> |
| <span class="source-line-no">485</span><span id="line-485"> * @throws BindException if the address can't be bound</span> |
| <span class="source-line-no">486</span><span id="line-486"> * @throws UnknownHostException if the address isn't a valid host name</span> |
| <span class="source-line-no">487</span><span id="line-487"> * @throws IOException other random errors from bind</span> |
| <span class="source-line-no">488</span><span id="line-488"> */</span> |
| <span class="source-line-no">489</span><span id="line-489"> public static void bind(ServerSocket socket, InetSocketAddress address, int backlog)</span> |
| <span class="source-line-no">490</span><span id="line-490"> throws IOException {</span> |
| <span class="source-line-no">491</span><span id="line-491"> try {</span> |
| <span class="source-line-no">492</span><span id="line-492"> socket.bind(address, backlog);</span> |
| <span class="source-line-no">493</span><span id="line-493"> } catch (BindException e) {</span> |
| <span class="source-line-no">494</span><span id="line-494"> BindException bindException =</span> |
| <span class="source-line-no">495</span><span id="line-495"> new BindException("Problem binding to " + address + " : " + e.getMessage());</span> |
| <span class="source-line-no">496</span><span id="line-496"> bindException.initCause(e);</span> |
| <span class="source-line-no">497</span><span id="line-497"> throw bindException;</span> |
| <span class="source-line-no">498</span><span id="line-498"> } catch (SocketException e) {</span> |
| <span class="source-line-no">499</span><span id="line-499"> // If they try to bind to a different host's address, give a better</span> |
| <span class="source-line-no">500</span><span id="line-500"> // error message.</span> |
| <span class="source-line-no">501</span><span id="line-501"> if ("Unresolved address".equals(e.getMessage())) {</span> |
| <span class="source-line-no">502</span><span id="line-502"> throw new UnknownHostException("Invalid hostname for server: " + address.getHostName());</span> |
| <span class="source-line-no">503</span><span id="line-503"> }</span> |
| <span class="source-line-no">504</span><span id="line-504"> throw e;</span> |
| <span class="source-line-no">505</span><span id="line-505"> }</span> |
| <span class="source-line-no">506</span><span id="line-506"> }</span> |
| <span class="source-line-no">507</span><span id="line-507"></span> |
| <span class="source-line-no">508</span><span id="line-508"> /**</span> |
| <span class="source-line-no">509</span><span id="line-509"> * The number of open RPC conections</span> |
| <span class="source-line-no">510</span><span id="line-510"> * @return the number of open rpc connections</span> |
| <span class="source-line-no">511</span><span id="line-511"> */</span> |
| <span class="source-line-no">512</span><span id="line-512"> @Override</span> |
| <span class="source-line-no">513</span><span id="line-513"> public int getNumOpenConnections() {</span> |
| <span class="source-line-no">514</span><span id="line-514"> return connectionManager.size();</span> |
| <span class="source-line-no">515</span><span id="line-515"> }</span> |
| <span class="source-line-no">516</span><span id="line-516"></span> |
| <span class="source-line-no">517</span><span id="line-517"> private class ConnectionManager {</span> |
| <span class="source-line-no">518</span><span id="line-518"> final private AtomicInteger count = new AtomicInteger();</span> |
| <span class="source-line-no">519</span><span id="line-519"> final private Set<SimpleServerRpcConnection> connections;</span> |
| <span class="source-line-no">520</span><span id="line-520"></span> |
| <span class="source-line-no">521</span><span id="line-521"> final private Timer idleScanTimer;</span> |
| <span class="source-line-no">522</span><span id="line-522"> final private int idleScanThreshold;</span> |
| <span class="source-line-no">523</span><span id="line-523"> final private int idleScanInterval;</span> |
| <span class="source-line-no">524</span><span id="line-524"> final private int maxIdleTime;</span> |
| <span class="source-line-no">525</span><span id="line-525"> final private int maxIdleToClose;</span> |
| <span class="source-line-no">526</span><span id="line-526"></span> |
| <span class="source-line-no">527</span><span id="line-527"> ConnectionManager() {</span> |
| <span class="source-line-no">528</span><span id="line-528"> this.idleScanTimer = new Timer("RpcServer idle connection scanner for port " + port, true);</span> |
| <span class="source-line-no">529</span><span id="line-529"> this.idleScanThreshold = conf.getInt("hbase.ipc.client.idlethreshold", 4000);</span> |
| <span class="source-line-no">530</span><span id="line-530"> this.idleScanInterval =</span> |
| <span class="source-line-no">531</span><span id="line-531"> conf.getInt("hbase.ipc.client.connection.idle-scan-interval.ms", 10000);</span> |
| <span class="source-line-no">532</span><span id="line-532"> this.maxIdleTime = 2 * conf.getInt("hbase.ipc.client.connection.maxidletime", 10000);</span> |
| <span class="source-line-no">533</span><span id="line-533"> this.maxIdleToClose = conf.getInt("hbase.ipc.client.kill.max", 10);</span> |
| <span class="source-line-no">534</span><span id="line-534"> int handlerCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT,</span> |
| <span class="source-line-no">535</span><span id="line-535"> HConstants.DEFAULT_REGION_SERVER_HANDLER_COUNT);</span> |
| <span class="source-line-no">536</span><span id="line-536"> int maxConnectionQueueSize =</span> |
| <span class="source-line-no">537</span><span id="line-537"> handlerCount * conf.getInt("hbase.ipc.server.handler.queue.size", 100);</span> |
| <span class="source-line-no">538</span><span id="line-538"> // create a set with concurrency -and- a thread-safe iterator, add 2</span> |
| <span class="source-line-no">539</span><span id="line-539"> // for listener and idle closer threads</span> |
| <span class="source-line-no">540</span><span id="line-540"> this.connections =</span> |
| <span class="source-line-no">541</span><span id="line-541"> Collections.newSetFromMap(new ConcurrentHashMap<SimpleServerRpcConnection, Boolean>(</span> |
| <span class="source-line-no">542</span><span id="line-542"> maxConnectionQueueSize, 0.75f, readThreads + 2));</span> |
| <span class="source-line-no">543</span><span id="line-543"> }</span> |
| <span class="source-line-no">544</span><span id="line-544"></span> |
| <span class="source-line-no">545</span><span id="line-545"> private boolean add(SimpleServerRpcConnection connection) {</span> |
| <span class="source-line-no">546</span><span id="line-546"> boolean added = connections.add(connection);</span> |
| <span class="source-line-no">547</span><span id="line-547"> if (added) {</span> |
| <span class="source-line-no">548</span><span id="line-548"> count.getAndIncrement();</span> |
| <span class="source-line-no">549</span><span id="line-549"> }</span> |
| <span class="source-line-no">550</span><span id="line-550"> return added;</span> |
| <span class="source-line-no">551</span><span id="line-551"> }</span> |
| <span class="source-line-no">552</span><span id="line-552"></span> |
| <span class="source-line-no">553</span><span id="line-553"> private boolean remove(SimpleServerRpcConnection connection) {</span> |
| <span class="source-line-no">554</span><span id="line-554"> boolean removed = connections.remove(connection);</span> |
| <span class="source-line-no">555</span><span id="line-555"> if (removed) {</span> |
| <span class="source-line-no">556</span><span id="line-556"> count.getAndDecrement();</span> |
| <span class="source-line-no">557</span><span id="line-557"> }</span> |
| <span class="source-line-no">558</span><span id="line-558"> return removed;</span> |
| <span class="source-line-no">559</span><span id="line-559"> }</span> |
| <span class="source-line-no">560</span><span id="line-560"></span> |
| <span class="source-line-no">561</span><span id="line-561"> int size() {</span> |
| <span class="source-line-no">562</span><span id="line-562"> return count.get();</span> |
| <span class="source-line-no">563</span><span id="line-563"> }</span> |
| <span class="source-line-no">564</span><span id="line-564"></span> |
| <span class="source-line-no">565</span><span id="line-565"> SimpleServerRpcConnection[] toArray() {</span> |
| <span class="source-line-no">566</span><span id="line-566"> return connections.toArray(new SimpleServerRpcConnection[0]);</span> |
| <span class="source-line-no">567</span><span id="line-567"> }</span> |
| <span class="source-line-no">568</span><span id="line-568"></span> |
| <span class="source-line-no">569</span><span id="line-569"> SimpleServerRpcConnection register(SocketChannel channel) {</span> |
| <span class="source-line-no">570</span><span id="line-570"> SimpleServerRpcConnection connection =</span> |
| <span class="source-line-no">571</span><span id="line-571"> getConnection(channel, EnvironmentEdgeManager.currentTime());</span> |
| <span class="source-line-no">572</span><span id="line-572"> add(connection);</span> |
| <span class="source-line-no">573</span><span id="line-573"> if (LOG.isTraceEnabled()) {</span> |
| <span class="source-line-no">574</span><span id="line-574"> LOG.trace("Connection from " + connection + "; connections=" + size()</span> |
| <span class="source-line-no">575</span><span id="line-575"> + ", queued calls size (bytes)=" + callQueueSizeInBytes.sum() + ", general queued calls="</span> |
| <span class="source-line-no">576</span><span id="line-576"> + scheduler.getGeneralQueueLength() + ", priority queued calls="</span> |
| <span class="source-line-no">577</span><span id="line-577"> + scheduler.getPriorityQueueLength() + ", meta priority queued calls="</span> |
| <span class="source-line-no">578</span><span id="line-578"> + scheduler.getMetaPriorityQueueLength());</span> |
| <span class="source-line-no">579</span><span id="line-579"> }</span> |
| <span class="source-line-no">580</span><span id="line-580"> return connection;</span> |
| <span class="source-line-no">581</span><span id="line-581"> }</span> |
| <span class="source-line-no">582</span><span id="line-582"></span> |
| <span class="source-line-no">583</span><span id="line-583"> boolean close(SimpleServerRpcConnection connection) {</span> |
| <span class="source-line-no">584</span><span id="line-584"> boolean exists = remove(connection);</span> |
| <span class="source-line-no">585</span><span id="line-585"> if (exists) {</span> |
| <span class="source-line-no">586</span><span id="line-586"> if (LOG.isTraceEnabled()) {</span> |
| <span class="source-line-no">587</span><span id="line-587"> LOG.trace(Thread.currentThread().getName() + ": disconnecting client " + connection</span> |
| <span class="source-line-no">588</span><span id="line-588"> + ". Number of active connections: " + size());</span> |
| <span class="source-line-no">589</span><span id="line-589"> }</span> |
| <span class="source-line-no">590</span><span id="line-590"> // only close if actually removed to avoid double-closing due</span> |
| <span class="source-line-no">591</span><span id="line-591"> // to possible races</span> |
| <span class="source-line-no">592</span><span id="line-592"> connection.close();</span> |
| <span class="source-line-no">593</span><span id="line-593"> }</span> |
| <span class="source-line-no">594</span><span id="line-594"> return exists;</span> |
| <span class="source-line-no">595</span><span id="line-595"> }</span> |
| <span class="source-line-no">596</span><span id="line-596"></span> |
| <span class="source-line-no">597</span><span id="line-597"> // synch'ed to avoid explicit invocation upon OOM from colliding with</span> |
| <span class="source-line-no">598</span><span id="line-598"> // timer task firing</span> |
| <span class="source-line-no">599</span><span id="line-599"> synchronized void closeIdle(boolean scanAll) {</span> |
| <span class="source-line-no">600</span><span id="line-600"> long minLastContact = EnvironmentEdgeManager.currentTime() - maxIdleTime;</span> |
| <span class="source-line-no">601</span><span id="line-601"> // concurrent iterator might miss new connections added</span> |
| <span class="source-line-no">602</span><span id="line-602"> // during the iteration, but that's ok because they won't</span> |
| <span class="source-line-no">603</span><span id="line-603"> // be idle yet anyway and will be caught on next scan</span> |
| <span class="source-line-no">604</span><span id="line-604"> int closed = 0;</span> |
| <span class="source-line-no">605</span><span id="line-605"> for (SimpleServerRpcConnection connection : connections) {</span> |
| <span class="source-line-no">606</span><span id="line-606"> // stop if connections dropped below threshold unless scanning all</span> |
| <span class="source-line-no">607</span><span id="line-607"> if (!scanAll && size() < idleScanThreshold) {</span> |
| <span class="source-line-no">608</span><span id="line-608"> break;</span> |
| <span class="source-line-no">609</span><span id="line-609"> }</span> |
| <span class="source-line-no">610</span><span id="line-610"> // stop if not scanning all and max connections are closed</span> |
| <span class="source-line-no">611</span><span id="line-611"> if (</span> |
| <span class="source-line-no">612</span><span id="line-612"> connection.isIdle() && connection.getLastContact() < minLastContact && close(connection)</span> |
| <span class="source-line-no">613</span><span id="line-613"> && !scanAll && (++closed == maxIdleToClose)</span> |
| <span class="source-line-no">614</span><span id="line-614"> ) {</span> |
| <span class="source-line-no">615</span><span id="line-615"> break;</span> |
| <span class="source-line-no">616</span><span id="line-616"> }</span> |
| <span class="source-line-no">617</span><span id="line-617"> }</span> |
| <span class="source-line-no">618</span><span id="line-618"> }</span> |
| <span class="source-line-no">619</span><span id="line-619"></span> |
| <span class="source-line-no">620</span><span id="line-620"> void closeAll() {</span> |
| <span class="source-line-no">621</span><span id="line-621"> // use a copy of the connections to be absolutely sure the concurrent</span> |
| <span class="source-line-no">622</span><span id="line-622"> // iterator doesn't miss a connection</span> |
| <span class="source-line-no">623</span><span id="line-623"> for (SimpleServerRpcConnection connection : toArray()) {</span> |
| <span class="source-line-no">624</span><span id="line-624"> close(connection);</span> |
| <span class="source-line-no">625</span><span id="line-625"> }</span> |
| <span class="source-line-no">626</span><span id="line-626"> }</span> |
| <span class="source-line-no">627</span><span id="line-627"></span> |
| <span class="source-line-no">628</span><span id="line-628"> void startIdleScan() {</span> |
| <span class="source-line-no">629</span><span id="line-629"> scheduleIdleScanTask();</span> |
| <span class="source-line-no">630</span><span id="line-630"> }</span> |
| <span class="source-line-no">631</span><span id="line-631"></span> |
| <span class="source-line-no">632</span><span id="line-632"> void stopIdleScan() {</span> |
| <span class="source-line-no">633</span><span id="line-633"> idleScanTimer.cancel();</span> |
| <span class="source-line-no">634</span><span id="line-634"> }</span> |
| <span class="source-line-no">635</span><span id="line-635"></span> |
| <span class="source-line-no">636</span><span id="line-636"> private void scheduleIdleScanTask() {</span> |
| <span class="source-line-no">637</span><span id="line-637"> if (!running) {</span> |
| <span class="source-line-no">638</span><span id="line-638"> return;</span> |
| <span class="source-line-no">639</span><span id="line-639"> }</span> |
| <span class="source-line-no">640</span><span id="line-640"> TimerTask idleScanTask = new TimerTask() {</span> |
| <span class="source-line-no">641</span><span id="line-641"> @Override</span> |
| <span class="source-line-no">642</span><span id="line-642"> public void run() {</span> |
| <span class="source-line-no">643</span><span id="line-643"> if (!running) {</span> |
| <span class="source-line-no">644</span><span id="line-644"> return;</span> |
| <span class="source-line-no">645</span><span id="line-645"> }</span> |
| <span class="source-line-no">646</span><span id="line-646"> if (LOG.isTraceEnabled()) {</span> |
| <span class="source-line-no">647</span><span id="line-647"> LOG.trace("running");</span> |
| <span class="source-line-no">648</span><span id="line-648"> }</span> |
| <span class="source-line-no">649</span><span id="line-649"> try {</span> |
| <span class="source-line-no">650</span><span id="line-650"> closeIdle(false);</span> |
| <span class="source-line-no">651</span><span id="line-651"> } finally {</span> |
| <span class="source-line-no">652</span><span id="line-652"> // explicitly reschedule so next execution occurs relative</span> |
| <span class="source-line-no">653</span><span id="line-653"> // to the end of this scan, not the beginning</span> |
| <span class="source-line-no">654</span><span id="line-654"> scheduleIdleScanTask();</span> |
| <span class="source-line-no">655</span><span id="line-655"> }</span> |
| <span class="source-line-no">656</span><span id="line-656"> }</span> |
| <span class="source-line-no">657</span><span id="line-657"> };</span> |
| <span class="source-line-no">658</span><span id="line-658"> idleScanTimer.schedule(idleScanTask, idleScanInterval);</span> |
| <span class="source-line-no">659</span><span id="line-659"> }</span> |
| <span class="source-line-no">660</span><span id="line-660"> }</span> |
| <span class="source-line-no">661</span><span id="line-661"></span> |
| <span class="source-line-no">662</span><span id="line-662">}</span> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| </pre> |
| </div> |
| </main> |
| </body> |
| </html> |