blob: e6a14f35db115a94d695393f4c849640403c3f6c [file] [log] [blame]
<!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.regionserver, class: BootstrapNodeManager">
<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.regionserver;</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.util.ArrayList;</span>
<span class="source-line-no">022</span><span id="line-22">import java.util.Collections;</span>
<span class="source-line-no">023</span><span id="line-23">import java.util.HashSet;</span>
<span class="source-line-no">024</span><span id="line-24">import java.util.List;</span>
<span class="source-line-no">025</span><span id="line-25">import java.util.Set;</span>
<span class="source-line-no">026</span><span id="line-26">import java.util.concurrent.Executors;</span>
<span class="source-line-no">027</span><span id="line-27">import java.util.concurrent.ScheduledExecutorService;</span>
<span class="source-line-no">028</span><span id="line-28">import java.util.concurrent.ThreadLocalRandom;</span>
<span class="source-line-no">029</span><span id="line-29">import java.util.concurrent.TimeUnit;</span>
<span class="source-line-no">030</span><span id="line-30">import java.util.stream.Collectors;</span>
<span class="source-line-no">031</span><span id="line-31">import org.apache.hadoop.conf.Configuration;</span>
<span class="source-line-no">032</span><span id="line-32">import org.apache.hadoop.hbase.HBaseRpcServicesBase;</span>
<span class="source-line-no">033</span><span id="line-33">import org.apache.hadoop.hbase.ServerName;</span>
<span class="source-line-no">034</span><span id="line-34">import org.apache.hadoop.hbase.client.AsyncClusterConnection;</span>
<span class="source-line-no">035</span><span id="line-35">import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;</span>
<span class="source-line-no">036</span><span id="line-36">import org.apache.hadoop.hbase.util.FutureUtils;</span>
<span class="source-line-no">037</span><span id="line-37">import org.apache.hadoop.hbase.util.RetryCounter;</span>
<span class="source-line-no">038</span><span id="line-38">import org.apache.hadoop.hbase.util.RetryCounter.ExponentialBackoffPolicyWithLimit;</span>
<span class="source-line-no">039</span><span id="line-39">import org.apache.hadoop.hbase.util.RetryCounter.RetryConfig;</span>
<span class="source-line-no">040</span><span id="line-40">import org.apache.hadoop.hbase.util.RetryCounterFactory;</span>
<span class="source-line-no">041</span><span id="line-41">import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;</span>
<span class="source-line-no">042</span><span id="line-42">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">043</span><span id="line-43">import org.slf4j.Logger;</span>
<span class="source-line-no">044</span><span id="line-44">import org.slf4j.LoggerFactory;</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.common.util.concurrent.ThreadFactoryBuilder;</span>
<span class="source-line-no">047</span><span id="line-47"></span>
<span class="source-line-no">048</span><span id="line-48">/**</span>
<span class="source-line-no">049</span><span id="line-49"> * Manage the bootstrap node list at region server side.</span>
<span class="source-line-no">050</span><span id="line-50"> * &lt;p/&gt;</span>
<span class="source-line-no">051</span><span id="line-51"> * It will request master first to get the initial set of bootstrap nodes(a sub set of live region</span>
<span class="source-line-no">052</span><span id="line-52"> * servers), and then it will exchange the bootstrap nodes with other bootstrap nodes. In most</span>
<span class="source-line-no">053</span><span id="line-53"> * cases, if the cluster is stable, we do not need to request master again until we reach the</span>
<span class="source-line-no">054</span><span id="line-54"> * request master interval. And if the current number of bootstrap nodes is not enough, we will</span>
<span class="source-line-no">055</span><span id="line-55"> * request master soon.</span>
<span class="source-line-no">056</span><span id="line-56"> * &lt;p/&gt;</span>
<span class="source-line-no">057</span><span id="line-57"> * The algorithm is very simple, as we will always fallback to request master. THe trick here is</span>
<span class="source-line-no">058</span><span id="line-58"> * that, if we can not get enough bootstrap nodes from master, then the cluster will be small, so it</span>
<span class="source-line-no">059</span><span id="line-59"> * will not put too much pressure on master if we always request master. And for large clusters, we</span>
<span class="source-line-no">060</span><span id="line-60"> * will soon get enough bootstrap nodes and stop requesting master.</span>
<span class="source-line-no">061</span><span id="line-61"> */</span>
<span class="source-line-no">062</span><span id="line-62">@InterfaceAudience.Private</span>
<span class="source-line-no">063</span><span id="line-63">public class BootstrapNodeManager {</span>
<span class="source-line-no">064</span><span id="line-64"></span>
<span class="source-line-no">065</span><span id="line-65"> private static final Logger LOG = LoggerFactory.getLogger(BootstrapNodeManager.class);</span>
<span class="source-line-no">066</span><span id="line-66"></span>
<span class="source-line-no">067</span><span id="line-67"> public static final String REQUEST_MASTER_INTERVAL_SECS =</span>
<span class="source-line-no">068</span><span id="line-68"> "hbase.server.bootstrap.request_master_interval.secs";</span>
<span class="source-line-no">069</span><span id="line-69"></span>
<span class="source-line-no">070</span><span id="line-70"> // default request every 10 minutes</span>
<span class="source-line-no">071</span><span id="line-71"> public static final long DEFAULT_REQUEST_MASTER_INTERVAL_SECS = TimeUnit.MINUTES.toSeconds(10);</span>
<span class="source-line-no">072</span><span id="line-72"></span>
<span class="source-line-no">073</span><span id="line-73"> public static final String REQUEST_MASTER_MIN_INTERVAL_SECS =</span>
<span class="source-line-no">074</span><span id="line-74"> "hbase.server.bootstrap.request_master_min_interval.secs";</span>
<span class="source-line-no">075</span><span id="line-75"></span>
<span class="source-line-no">076</span><span id="line-76"> // default 30 seconds</span>
<span class="source-line-no">077</span><span id="line-77"> public static final long DEFAULT_REQUEST_MASTER_MIN_INTERVAL_SECS = 30;</span>
<span class="source-line-no">078</span><span id="line-78"></span>
<span class="source-line-no">079</span><span id="line-79"> public static final String REQUEST_REGIONSERVER_INTERVAL_SECS =</span>
<span class="source-line-no">080</span><span id="line-80"> "hbase.server.bootstrap.request_regionserver_interval.secs";</span>
<span class="source-line-no">081</span><span id="line-81"></span>
<span class="source-line-no">082</span><span id="line-82"> // default request every 30 seconds</span>
<span class="source-line-no">083</span><span id="line-83"> public static final long DEFAULT_REQUEST_REGIONSERVER_INTERVAL_SECS = 30;</span>
<span class="source-line-no">084</span><span id="line-84"></span>
<span class="source-line-no">085</span><span id="line-85"> private static final float JITTER = 0.2f;</span>
<span class="source-line-no">086</span><span id="line-86"></span>
<span class="source-line-no">087</span><span id="line-87"> private volatile List&lt;ServerName&gt; nodes = Collections.emptyList();</span>
<span class="source-line-no">088</span><span id="line-88"></span>
<span class="source-line-no">089</span><span id="line-89"> private final AsyncClusterConnection conn;</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 final MasterAddressTracker masterAddrTracker;</span>
<span class="source-line-no">092</span><span id="line-92"></span>
<span class="source-line-no">093</span><span id="line-93"> private final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(</span>
<span class="source-line-no">094</span><span id="line-94"> new ThreadFactoryBuilder().setDaemon(true).setNameFormat(getClass().getSimpleName()).build());</span>
<span class="source-line-no">095</span><span id="line-95"></span>
<span class="source-line-no">096</span><span id="line-96"> private final long requestMasterIntervalSecs;</span>
<span class="source-line-no">097</span><span id="line-97"></span>
<span class="source-line-no">098</span><span id="line-98"> private final long requestMasterMinIntervalSecs;</span>
<span class="source-line-no">099</span><span id="line-99"></span>
<span class="source-line-no">100</span><span id="line-100"> private final long requestRegionServerIntervalSecs;</span>
<span class="source-line-no">101</span><span id="line-101"></span>
<span class="source-line-no">102</span><span id="line-102"> private final int maxNodeCount;</span>
<span class="source-line-no">103</span><span id="line-103"></span>
<span class="source-line-no">104</span><span id="line-104"> private final RetryCounterFactory retryCounterFactory;</span>
<span class="source-line-no">105</span><span id="line-105"></span>
<span class="source-line-no">106</span><span id="line-106"> private RetryCounter retryCounter;</span>
<span class="source-line-no">107</span><span id="line-107"></span>
<span class="source-line-no">108</span><span id="line-108"> private long lastRequestMasterTime;</span>
<span class="source-line-no">109</span><span id="line-109"></span>
<span class="source-line-no">110</span><span id="line-110"> public BootstrapNodeManager(AsyncClusterConnection conn, MasterAddressTracker masterAddrTracker) {</span>
<span class="source-line-no">111</span><span id="line-111"> this.conn = conn;</span>
<span class="source-line-no">112</span><span id="line-112"> this.masterAddrTracker = masterAddrTracker;</span>
<span class="source-line-no">113</span><span id="line-113"> Configuration conf = conn.getConfiguration();</span>
<span class="source-line-no">114</span><span id="line-114"> requestMasterIntervalSecs =</span>
<span class="source-line-no">115</span><span id="line-115"> conf.getLong(REQUEST_MASTER_INTERVAL_SECS, DEFAULT_REQUEST_MASTER_INTERVAL_SECS);</span>
<span class="source-line-no">116</span><span id="line-116"> requestMasterMinIntervalSecs =</span>
<span class="source-line-no">117</span><span id="line-117"> conf.getLong(REQUEST_MASTER_MIN_INTERVAL_SECS, DEFAULT_REQUEST_MASTER_MIN_INTERVAL_SECS);</span>
<span class="source-line-no">118</span><span id="line-118"> requestRegionServerIntervalSecs =</span>
<span class="source-line-no">119</span><span id="line-119"> conf.getLong(REQUEST_REGIONSERVER_INTERVAL_SECS, DEFAULT_REQUEST_REGIONSERVER_INTERVAL_SECS);</span>
<span class="source-line-no">120</span><span id="line-120"> maxNodeCount = conf.getInt(HBaseRpcServicesBase.CLIENT_BOOTSTRAP_NODE_LIMIT,</span>
<span class="source-line-no">121</span><span id="line-121"> HBaseRpcServicesBase.DEFAULT_CLIENT_BOOTSTRAP_NODE_LIMIT);</span>
<span class="source-line-no">122</span><span id="line-122"> retryCounterFactory = new RetryCounterFactory(</span>
<span class="source-line-no">123</span><span id="line-123"> new RetryConfig().setBackoffPolicy(new ExponentialBackoffPolicyWithLimit()).setJitter(JITTER)</span>
<span class="source-line-no">124</span><span id="line-124"> .setSleepInterval(requestMasterMinIntervalSecs).setMaxSleepTime(requestMasterIntervalSecs)</span>
<span class="source-line-no">125</span><span id="line-125"> .setTimeUnit(TimeUnit.SECONDS));</span>
<span class="source-line-no">126</span><span id="line-126"> executor.schedule(this::getFromMaster, getDelay(requestMasterMinIntervalSecs),</span>
<span class="source-line-no">127</span><span id="line-127"> TimeUnit.SECONDS);</span>
<span class="source-line-no">128</span><span id="line-128"> }</span>
<span class="source-line-no">129</span><span id="line-129"></span>
<span class="source-line-no">130</span><span id="line-130"> private long getDelay(long delay) {</span>
<span class="source-line-no">131</span><span id="line-131"> long jitterDelay = (long) (delay * ThreadLocalRandom.current().nextFloat() * JITTER);</span>
<span class="source-line-no">132</span><span id="line-132"> return delay + jitterDelay;</span>
<span class="source-line-no">133</span><span id="line-133"> }</span>
<span class="source-line-no">134</span><span id="line-134"></span>
<span class="source-line-no">135</span><span id="line-135"> private void getFromMaster() {</span>
<span class="source-line-no">136</span><span id="line-136"> List&lt;ServerName&gt; liveRegionServers;</span>
<span class="source-line-no">137</span><span id="line-137"> try {</span>
<span class="source-line-no">138</span><span id="line-138"> // get 2 times number of node</span>
<span class="source-line-no">139</span><span id="line-139"> liveRegionServers =</span>
<span class="source-line-no">140</span><span id="line-140"> FutureUtils.get(conn.getLiveRegionServers(masterAddrTracker, maxNodeCount * 2));</span>
<span class="source-line-no">141</span><span id="line-141"> } catch (IOException e) {</span>
<span class="source-line-no">142</span><span id="line-142"> LOG.warn("failed to get live region servers from master", e);</span>
<span class="source-line-no">143</span><span id="line-143"> if (retryCounter == null) {</span>
<span class="source-line-no">144</span><span id="line-144"> retryCounter = retryCounterFactory.create();</span>
<span class="source-line-no">145</span><span id="line-145"> }</span>
<span class="source-line-no">146</span><span id="line-146"> executor.schedule(this::getFromMaster, retryCounter.getBackoffTimeAndIncrementAttempts(),</span>
<span class="source-line-no">147</span><span id="line-147"> TimeUnit.SECONDS);</span>
<span class="source-line-no">148</span><span id="line-148"> return;</span>
<span class="source-line-no">149</span><span id="line-149"> }</span>
<span class="source-line-no">150</span><span id="line-150"> retryCounter = null;</span>
<span class="source-line-no">151</span><span id="line-151"> lastRequestMasterTime = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">152</span><span id="line-152"> this.nodes = Collections.unmodifiableList(liveRegionServers);</span>
<span class="source-line-no">153</span><span id="line-153"> if (liveRegionServers.size() &lt; maxNodeCount) {</span>
<span class="source-line-no">154</span><span id="line-154"> // If the number of live region servers is small, it means the cluster is small, so requesting</span>
<span class="source-line-no">155</span><span id="line-155"> // master with a higher frequency will not be a big problem, so here we will always request</span>
<span class="source-line-no">156</span><span id="line-156"> // master to get the live region servers as bootstrap nodes.</span>
<span class="source-line-no">157</span><span id="line-157"> executor.schedule(this::getFromMaster, getDelay(requestMasterMinIntervalSecs),</span>
<span class="source-line-no">158</span><span id="line-158"> TimeUnit.SECONDS);</span>
<span class="source-line-no">159</span><span id="line-159"> return;</span>
<span class="source-line-no">160</span><span id="line-160"> }</span>
<span class="source-line-no">161</span><span id="line-161"> // schedule tasks to exchange the bootstrap nodes with other region servers.</span>
<span class="source-line-no">162</span><span id="line-162"> executor.schedule(this::getFromRegionServer, getDelay(requestRegionServerIntervalSecs),</span>
<span class="source-line-no">163</span><span id="line-163"> TimeUnit.SECONDS);</span>
<span class="source-line-no">164</span><span id="line-164"> }</span>
<span class="source-line-no">165</span><span id="line-165"></span>
<span class="source-line-no">166</span><span id="line-166"> // this method is also used to test whether a given region server is still alive.</span>
<span class="source-line-no">167</span><span id="line-167"> private void getFromRegionServer() {</span>
<span class="source-line-no">168</span><span id="line-168"> if (</span>
<span class="source-line-no">169</span><span id="line-169"> EnvironmentEdgeManager.currentTime() - lastRequestMasterTime</span>
<span class="source-line-no">170</span><span id="line-170"> &gt;= TimeUnit.SECONDS.toMillis(requestMasterIntervalSecs)</span>
<span class="source-line-no">171</span><span id="line-171"> ) {</span>
<span class="source-line-no">172</span><span id="line-172"> // schedule a get from master task immediately if haven't request master for more than</span>
<span class="source-line-no">173</span><span id="line-173"> // requestMasterIntervalSecs</span>
<span class="source-line-no">174</span><span id="line-174"> executor.execute(this::getFromMaster);</span>
<span class="source-line-no">175</span><span id="line-175"> return;</span>
<span class="source-line-no">176</span><span id="line-176"> }</span>
<span class="source-line-no">177</span><span id="line-177"> List&lt;ServerName&gt; currentList = this.nodes;</span>
<span class="source-line-no">178</span><span id="line-178"> ServerName peer = currentList.get(ThreadLocalRandom.current().nextInt(currentList.size()));</span>
<span class="source-line-no">179</span><span id="line-179"> List&lt;ServerName&gt; otherList;</span>
<span class="source-line-no">180</span><span id="line-180"> try {</span>
<span class="source-line-no">181</span><span id="line-181"> otherList = FutureUtils.get(conn.getAllBootstrapNodes(peer));</span>
<span class="source-line-no">182</span><span id="line-182"> } catch (IOException e) {</span>
<span class="source-line-no">183</span><span id="line-183"> LOG.warn("failed to request region server {}", peer, e);</span>
<span class="source-line-no">184</span><span id="line-184"> // remove this region server from the list since it can not respond successfully</span>
<span class="source-line-no">185</span><span id="line-185"> List&lt;ServerName&gt; newList = currentList.stream().filter(sn -&gt; sn != peer)</span>
<span class="source-line-no">186</span><span id="line-186"> .collect(Collectors.collectingAndThen(Collectors.toList(), Collections::unmodifiableList));</span>
<span class="source-line-no">187</span><span id="line-187"> this.nodes = newList;</span>
<span class="source-line-no">188</span><span id="line-188"> if (newList.size() &lt; maxNodeCount) {</span>
<span class="source-line-no">189</span><span id="line-189"> // schedule a get from master task immediately</span>
<span class="source-line-no">190</span><span id="line-190"> executor.execute(this::getFromMaster);</span>
<span class="source-line-no">191</span><span id="line-191"> } else {</span>
<span class="source-line-no">192</span><span id="line-192"> executor.schedule(this::getFromRegionServer, getDelay(requestRegionServerIntervalSecs),</span>
<span class="source-line-no">193</span><span id="line-193"> TimeUnit.SECONDS);</span>
<span class="source-line-no">194</span><span id="line-194"> }</span>
<span class="source-line-no">195</span><span id="line-195"> return;</span>
<span class="source-line-no">196</span><span id="line-196"> }</span>
<span class="source-line-no">197</span><span id="line-197"> // randomly select new live region server list</span>
<span class="source-line-no">198</span><span id="line-198"> Set&lt;ServerName&gt; newRegionServers = new HashSet&lt;ServerName&gt;(currentList);</span>
<span class="source-line-no">199</span><span id="line-199"> newRegionServers.addAll(otherList);</span>
<span class="source-line-no">200</span><span id="line-200"> List&lt;ServerName&gt; newList = new ArrayList&lt;ServerName&gt;(newRegionServers);</span>
<span class="source-line-no">201</span><span id="line-201"> Collections.shuffle(newList, ThreadLocalRandom.current());</span>
<span class="source-line-no">202</span><span id="line-202"> int expectedListSize = maxNodeCount * 2;</span>
<span class="source-line-no">203</span><span id="line-203"> if (newList.size() &lt;= expectedListSize) {</span>
<span class="source-line-no">204</span><span id="line-204"> this.nodes = Collections.unmodifiableList(newList);</span>
<span class="source-line-no">205</span><span id="line-205"> } else {</span>
<span class="source-line-no">206</span><span id="line-206"> this.nodes =</span>
<span class="source-line-no">207</span><span id="line-207"> Collections.unmodifiableList(new ArrayList&lt;&gt;(newList.subList(0, expectedListSize)));</span>
<span class="source-line-no">208</span><span id="line-208"> }</span>
<span class="source-line-no">209</span><span id="line-209"> // schedule a new get from region server task</span>
<span class="source-line-no">210</span><span id="line-210"> executor.schedule(this::getFromRegionServer, requestRegionServerIntervalSecs, TimeUnit.SECONDS);</span>
<span class="source-line-no">211</span><span id="line-211"> }</span>
<span class="source-line-no">212</span><span id="line-212"></span>
<span class="source-line-no">213</span><span id="line-213"> public void stop() {</span>
<span class="source-line-no">214</span><span id="line-214"> executor.shutdownNow();</span>
<span class="source-line-no">215</span><span id="line-215"> }</span>
<span class="source-line-no">216</span><span id="line-216"></span>
<span class="source-line-no">217</span><span id="line-217"> public List&lt;ServerName&gt; getBootstrapNodes() {</span>
<span class="source-line-no">218</span><span id="line-218"> return nodes;</span>
<span class="source-line-no">219</span><span id="line-219"> }</span>
<span class="source-line-no">220</span><span id="line-220">}</span>
</pre>
</div>
</main>
</body>
</html>