blob: 8eab4992c4f3cb0f5d1ccc1617bef3ca55ea6b51 [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.client, class: AsyncConnectionImpl">
<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.client;</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 static org.apache.hadoop.hbase.HConstants.STATUS_PUBLISHED;</span>
<span class="source-line-no">021</span><span id="line-21">import static org.apache.hadoop.hbase.HConstants.STATUS_PUBLISHED_DEFAULT;</span>
<span class="source-line-no">022</span><span id="line-22">import static org.apache.hadoop.hbase.client.ClusterStatusListener.DEFAULT_STATUS_LISTENER_CLASS;</span>
<span class="source-line-no">023</span><span id="line-23">import static org.apache.hadoop.hbase.client.ClusterStatusListener.STATUS_LISTENER_CLASS;</span>
<span class="source-line-no">024</span><span id="line-24">import static org.apache.hadoop.hbase.client.ConnectionUtils.NO_NONCE_GENERATOR;</span>
<span class="source-line-no">025</span><span id="line-25">import static org.apache.hadoop.hbase.client.ConnectionUtils.getStubKey;</span>
<span class="source-line-no">026</span><span id="line-26">import static org.apache.hadoop.hbase.client.MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY;</span>
<span class="source-line-no">027</span><span id="line-27">import static org.apache.hadoop.hbase.client.NonceGenerator.CLIENT_NONCES_ENABLED_KEY;</span>
<span class="source-line-no">028</span><span id="line-28">import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.SERVER_NAME_KEY;</span>
<span class="source-line-no">029</span><span id="line-29">import static org.apache.hadoop.hbase.util.FutureUtils.addListener;</span>
<span class="source-line-no">030</span><span id="line-30"></span>
<span class="source-line-no">031</span><span id="line-31">import io.opentelemetry.api.trace.Span;</span>
<span class="source-line-no">032</span><span id="line-32">import java.io.IOException;</span>
<span class="source-line-no">033</span><span id="line-33">import java.net.SocketAddress;</span>
<span class="source-line-no">034</span><span id="line-34">import java.util.Collections;</span>
<span class="source-line-no">035</span><span id="line-35">import java.util.Map;</span>
<span class="source-line-no">036</span><span id="line-36">import java.util.Optional;</span>
<span class="source-line-no">037</span><span id="line-37">import java.util.concurrent.CompletableFuture;</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.ConcurrentMap;</span>
<span class="source-line-no">040</span><span id="line-40">import java.util.concurrent.ExecutionException;</span>
<span class="source-line-no">041</span><span id="line-41">import java.util.concurrent.ExecutorService;</span>
<span class="source-line-no">042</span><span id="line-42">import java.util.concurrent.TimeUnit;</span>
<span class="source-line-no">043</span><span id="line-43">import java.util.concurrent.atomic.AtomicBoolean;</span>
<span class="source-line-no">044</span><span id="line-44">import java.util.concurrent.atomic.AtomicReference;</span>
<span class="source-line-no">045</span><span id="line-45">import org.apache.commons.io.IOUtils;</span>
<span class="source-line-no">046</span><span id="line-46">import org.apache.hadoop.conf.Configuration;</span>
<span class="source-line-no">047</span><span id="line-47">import org.apache.hadoop.hbase.AuthUtil;</span>
<span class="source-line-no">048</span><span id="line-48">import org.apache.hadoop.hbase.ChoreService;</span>
<span class="source-line-no">049</span><span id="line-49">import org.apache.hadoop.hbase.MasterNotRunningException;</span>
<span class="source-line-no">050</span><span id="line-50">import org.apache.hadoop.hbase.ServerName;</span>
<span class="source-line-no">051</span><span id="line-51">import org.apache.hadoop.hbase.TableName;</span>
<span class="source-line-no">052</span><span id="line-52">import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;</span>
<span class="source-line-no">053</span><span id="line-53">import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicyFactory;</span>
<span class="source-line-no">054</span><span id="line-54">import org.apache.hadoop.hbase.ipc.RpcClient;</span>
<span class="source-line-no">055</span><span id="line-55">import org.apache.hadoop.hbase.ipc.RpcClientFactory;</span>
<span class="source-line-no">056</span><span id="line-56">import org.apache.hadoop.hbase.ipc.RpcControllerFactory;</span>
<span class="source-line-no">057</span><span id="line-57">import org.apache.hadoop.hbase.security.User;</span>
<span class="source-line-no">058</span><span id="line-58">import org.apache.hadoop.hbase.trace.TraceUtil;</span>
<span class="source-line-no">059</span><span id="line-59">import org.apache.hadoop.hbase.util.ConcurrentMapUtils;</span>
<span class="source-line-no">060</span><span id="line-60">import org.apache.hadoop.hbase.util.Threads;</span>
<span class="source-line-no">061</span><span id="line-61">import org.apache.hadoop.security.UserGroupInformation;</span>
<span class="source-line-no">062</span><span id="line-62">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">063</span><span id="line-63">import org.slf4j.Logger;</span>
<span class="source-line-no">064</span><span id="line-64">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">065</span><span id="line-65"></span>
<span class="source-line-no">066</span><span id="line-66">import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;</span>
<span class="source-line-no">067</span><span id="line-67">import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;</span>
<span class="source-line-no">068</span><span id="line-68"></span>
<span class="source-line-no">069</span><span id="line-69">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;</span>
<span class="source-line-no">070</span><span id="line-70">import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;</span>
<span class="source-line-no">071</span><span id="line-71">import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;</span>
<span class="source-line-no">072</span><span id="line-72">import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;</span>
<span class="source-line-no">073</span><span id="line-73"></span>
<span class="source-line-no">074</span><span id="line-74">/**</span>
<span class="source-line-no">075</span><span id="line-75"> * The implementation of AsyncConnection.</span>
<span class="source-line-no">076</span><span id="line-76"> */</span>
<span class="source-line-no">077</span><span id="line-77">@InterfaceAudience.Private</span>
<span class="source-line-no">078</span><span id="line-78">public class AsyncConnectionImpl implements AsyncConnection {</span>
<span class="source-line-no">079</span><span id="line-79"></span>
<span class="source-line-no">080</span><span id="line-80"> private static final Logger LOG = LoggerFactory.getLogger(AsyncConnectionImpl.class);</span>
<span class="source-line-no">081</span><span id="line-81"></span>
<span class="source-line-no">082</span><span id="line-82"> static final HashedWheelTimer RETRY_TIMER = new HashedWheelTimer(</span>
<span class="source-line-no">083</span><span id="line-83"> new ThreadFactoryBuilder().setNameFormat("Async-Client-Retry-Timer-pool-%d").setDaemon(true)</span>
<span class="source-line-no">084</span><span id="line-84"> .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build(),</span>
<span class="source-line-no">085</span><span id="line-85"> 10, TimeUnit.MILLISECONDS);</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 final Configuration conf;</span>
<span class="source-line-no">088</span><span id="line-88"></span>
<span class="source-line-no">089</span><span id="line-89"> final AsyncConnectionConfiguration connConf;</span>
<span class="source-line-no">090</span><span id="line-90"></span>
<span class="source-line-no">091</span><span id="line-91"> protected final User user;</span>
<span class="source-line-no">092</span><span id="line-92"></span>
<span class="source-line-no">093</span><span id="line-93"> final ConnectionRegistry registry;</span>
<span class="source-line-no">094</span><span id="line-94"></span>
<span class="source-line-no">095</span><span id="line-95"> protected final int rpcTimeout;</span>
<span class="source-line-no">096</span><span id="line-96"></span>
<span class="source-line-no">097</span><span id="line-97"> protected final RpcClient rpcClient;</span>
<span class="source-line-no">098</span><span id="line-98"></span>
<span class="source-line-no">099</span><span id="line-99"> final RpcControllerFactory rpcControllerFactory;</span>
<span class="source-line-no">100</span><span id="line-100"></span>
<span class="source-line-no">101</span><span id="line-101"> private final AsyncRegionLocator locator;</span>
<span class="source-line-no">102</span><span id="line-102"></span>
<span class="source-line-no">103</span><span id="line-103"> final AsyncRpcRetryingCallerFactory callerFactory;</span>
<span class="source-line-no">104</span><span id="line-104"></span>
<span class="source-line-no">105</span><span id="line-105"> private final NonceGenerator nonceGenerator;</span>
<span class="source-line-no">106</span><span id="line-106"></span>
<span class="source-line-no">107</span><span id="line-107"> private final ConcurrentMap&lt;String, ClientService.Interface&gt; rsStubs = new ConcurrentHashMap&lt;&gt;();</span>
<span class="source-line-no">108</span><span id="line-108"> private final ConcurrentMap&lt;String, AdminService.Interface&gt; adminStubs =</span>
<span class="source-line-no">109</span><span id="line-109"> new ConcurrentHashMap&lt;&gt;();</span>
<span class="source-line-no">110</span><span id="line-110"></span>
<span class="source-line-no">111</span><span id="line-111"> private final AtomicReference&lt;MasterService.Interface&gt; masterStub = new AtomicReference&lt;&gt;();</span>
<span class="source-line-no">112</span><span id="line-112"></span>
<span class="source-line-no">113</span><span id="line-113"> private final AtomicReference&lt;CompletableFuture&lt;MasterService.Interface&gt;&gt; masterStubMakeFuture =</span>
<span class="source-line-no">114</span><span id="line-114"> new AtomicReference&lt;&gt;();</span>
<span class="source-line-no">115</span><span id="line-115"></span>
<span class="source-line-no">116</span><span id="line-116"> private final Optional&lt;ServerStatisticTracker&gt; stats;</span>
<span class="source-line-no">117</span><span id="line-117"> private final ClientBackoffPolicy backoffPolicy;</span>
<span class="source-line-no">118</span><span id="line-118"></span>
<span class="source-line-no">119</span><span id="line-119"> private ChoreService choreService;</span>
<span class="source-line-no">120</span><span id="line-120"></span>
<span class="source-line-no">121</span><span id="line-121"> private final AtomicBoolean closed = new AtomicBoolean(false);</span>
<span class="source-line-no">122</span><span id="line-122"></span>
<span class="source-line-no">123</span><span id="line-123"> private final String metricsScope;</span>
<span class="source-line-no">124</span><span id="line-124"> private final Optional&lt;MetricsConnection&gt; metrics;</span>
<span class="source-line-no">125</span><span id="line-125"></span>
<span class="source-line-no">126</span><span id="line-126"> private final ClusterStatusListener clusterStatusListener;</span>
<span class="source-line-no">127</span><span id="line-127"></span>
<span class="source-line-no">128</span><span id="line-128"> private volatile ConnectionOverAsyncConnection conn;</span>
<span class="source-line-no">129</span><span id="line-129"></span>
<span class="source-line-no">130</span><span id="line-130"> public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, String clusterId,</span>
<span class="source-line-no">131</span><span id="line-131"> SocketAddress localAddress, User user) {</span>
<span class="source-line-no">132</span><span id="line-132"> this(conf, registry, clusterId, localAddress, user, Collections.emptyMap());</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"> public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, String clusterId,</span>
<span class="source-line-no">136</span><span id="line-136"> SocketAddress localAddress, User user, Map&lt;String, byte[]&gt; connectionAttributes) {</span>
<span class="source-line-no">137</span><span id="line-137"> this.conf = conf;</span>
<span class="source-line-no">138</span><span id="line-138"> this.user = user;</span>
<span class="source-line-no">139</span><span id="line-139"> this.metricsScope = MetricsConnection.getScope(conf, clusterId, this);</span>
<span class="source-line-no">140</span><span id="line-140"></span>
<span class="source-line-no">141</span><span id="line-141"> if (user.isLoginFromKeytab()) {</span>
<span class="source-line-no">142</span><span id="line-142"> spawnRenewalChore(user.getUGI());</span>
<span class="source-line-no">143</span><span id="line-143"> }</span>
<span class="source-line-no">144</span><span id="line-144"> this.connConf = new AsyncConnectionConfiguration(conf);</span>
<span class="source-line-no">145</span><span id="line-145"> this.registry = registry;</span>
<span class="source-line-no">146</span><span id="line-146"> if (conf.getBoolean(CLIENT_SIDE_METRICS_ENABLED_KEY, false)) {</span>
<span class="source-line-no">147</span><span id="line-147"> this.metrics = Optional</span>
<span class="source-line-no">148</span><span id="line-148"> .of(MetricsConnection.getMetricsConnection(conf, metricsScope, () -&gt; null, () -&gt; null));</span>
<span class="source-line-no">149</span><span id="line-149"> } else {</span>
<span class="source-line-no">150</span><span id="line-150"> this.metrics = Optional.empty();</span>
<span class="source-line-no">151</span><span id="line-151"> }</span>
<span class="source-line-no">152</span><span id="line-152"> this.rpcClient = RpcClientFactory.createClient(conf, clusterId, localAddress,</span>
<span class="source-line-no">153</span><span id="line-153"> metrics.orElse(null), connectionAttributes);</span>
<span class="source-line-no">154</span><span id="line-154"> this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);</span>
<span class="source-line-no">155</span><span id="line-155"> this.rpcTimeout =</span>
<span class="source-line-no">156</span><span id="line-156"> (int) Math.min(Integer.MAX_VALUE, TimeUnit.NANOSECONDS.toMillis(connConf.getRpcTimeoutNs()));</span>
<span class="source-line-no">157</span><span id="line-157"> this.locator = new AsyncRegionLocator(this, RETRY_TIMER);</span>
<span class="source-line-no">158</span><span id="line-158"> this.callerFactory = new AsyncRpcRetryingCallerFactory(this, RETRY_TIMER);</span>
<span class="source-line-no">159</span><span id="line-159"> if (conf.getBoolean(CLIENT_NONCES_ENABLED_KEY, true)) {</span>
<span class="source-line-no">160</span><span id="line-160"> nonceGenerator = PerClientRandomNonceGenerator.get();</span>
<span class="source-line-no">161</span><span id="line-161"> } else {</span>
<span class="source-line-no">162</span><span id="line-162"> nonceGenerator = NO_NONCE_GENERATOR;</span>
<span class="source-line-no">163</span><span id="line-163"> }</span>
<span class="source-line-no">164</span><span id="line-164"> this.stats = Optional.ofNullable(ServerStatisticTracker.create(conf));</span>
<span class="source-line-no">165</span><span id="line-165"> this.backoffPolicy = ClientBackoffPolicyFactory.create(conf);</span>
<span class="source-line-no">166</span><span id="line-166"> ClusterStatusListener listener = null;</span>
<span class="source-line-no">167</span><span id="line-167"> if (conf.getBoolean(STATUS_PUBLISHED, STATUS_PUBLISHED_DEFAULT)) {</span>
<span class="source-line-no">168</span><span id="line-168"> // TODO: this maybe a blocking operation, better to create it outside the constructor and pass</span>
<span class="source-line-no">169</span><span id="line-169"> // it in, just like clusterId. Not a big problem for now as the default value is false.</span>
<span class="source-line-no">170</span><span id="line-170"> Class&lt;? extends ClusterStatusListener.Listener&gt; listenerClass = conf.getClass(</span>
<span class="source-line-no">171</span><span id="line-171"> STATUS_LISTENER_CLASS, DEFAULT_STATUS_LISTENER_CLASS, ClusterStatusListener.Listener.class);</span>
<span class="source-line-no">172</span><span id="line-172"> if (listenerClass == null) {</span>
<span class="source-line-no">173</span><span id="line-173"> LOG.warn("{} is true, but {} is not set", STATUS_PUBLISHED, STATUS_LISTENER_CLASS);</span>
<span class="source-line-no">174</span><span id="line-174"> } else {</span>
<span class="source-line-no">175</span><span id="line-175"> try {</span>
<span class="source-line-no">176</span><span id="line-176"> listener = new ClusterStatusListener(new ClusterStatusListener.DeadServerHandler() {</span>
<span class="source-line-no">177</span><span id="line-177"> @Override</span>
<span class="source-line-no">178</span><span id="line-178"> public void newDead(ServerName sn) {</span>
<span class="source-line-no">179</span><span id="line-179"> locator.clearCache(sn);</span>
<span class="source-line-no">180</span><span id="line-180"> rpcClient.cancelConnections(sn);</span>
<span class="source-line-no">181</span><span id="line-181"> }</span>
<span class="source-line-no">182</span><span id="line-182"> }, conf, listenerClass);</span>
<span class="source-line-no">183</span><span id="line-183"> } catch (IOException e) {</span>
<span class="source-line-no">184</span><span id="line-184"> LOG.warn("Failed create of ClusterStatusListener, not a critical, ignoring...", e);</span>
<span class="source-line-no">185</span><span id="line-185"> }</span>
<span class="source-line-no">186</span><span id="line-186"> }</span>
<span class="source-line-no">187</span><span id="line-187"> }</span>
<span class="source-line-no">188</span><span id="line-188"> this.clusterStatusListener = listener;</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"> private void spawnRenewalChore(final UserGroupInformation user) {</span>
<span class="source-line-no">192</span><span id="line-192"> ChoreService service = getChoreService();</span>
<span class="source-line-no">193</span><span id="line-193"> service.scheduleChore(AuthUtil.getAuthRenewalChore(user, conf));</span>
<span class="source-line-no">194</span><span id="line-194"> }</span>
<span class="source-line-no">195</span><span id="line-195"></span>
<span class="source-line-no">196</span><span id="line-196"> /**</span>
<span class="source-line-no">197</span><span id="line-197"> * If choreService has not been created yet, create the ChoreService.</span>
<span class="source-line-no">198</span><span id="line-198"> */</span>
<span class="source-line-no">199</span><span id="line-199"> synchronized ChoreService getChoreService() {</span>
<span class="source-line-no">200</span><span id="line-200"> if (isClosed()) {</span>
<span class="source-line-no">201</span><span id="line-201"> throw new IllegalStateException("connection is already closed");</span>
<span class="source-line-no">202</span><span id="line-202"> }</span>
<span class="source-line-no">203</span><span id="line-203"> if (choreService == null) {</span>
<span class="source-line-no">204</span><span id="line-204"> choreService = new ChoreService("AsyncConn Chore Service");</span>
<span class="source-line-no">205</span><span id="line-205"> }</span>
<span class="source-line-no">206</span><span id="line-206"> return choreService;</span>
<span class="source-line-no">207</span><span id="line-207"> }</span>
<span class="source-line-no">208</span><span id="line-208"></span>
<span class="source-line-no">209</span><span id="line-209"> public User getUser() {</span>
<span class="source-line-no">210</span><span id="line-210"> return user;</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 ConnectionRegistry getConnectionRegistry() {</span>
<span class="source-line-no">214</span><span id="line-214"> return registry;</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"> @Override</span>
<span class="source-line-no">218</span><span id="line-218"> public Configuration getConfiguration() {</span>
<span class="source-line-no">219</span><span id="line-219"> return conf;</span>
<span class="source-line-no">220</span><span id="line-220"> }</span>
<span class="source-line-no">221</span><span id="line-221"></span>
<span class="source-line-no">222</span><span id="line-222"> @Override</span>
<span class="source-line-no">223</span><span id="line-223"> public boolean isClosed() {</span>
<span class="source-line-no">224</span><span id="line-224"> return closed.get();</span>
<span class="source-line-no">225</span><span id="line-225"> }</span>
<span class="source-line-no">226</span><span id="line-226"></span>
<span class="source-line-no">227</span><span id="line-227"> @Override</span>
<span class="source-line-no">228</span><span id="line-228"> public void close() {</span>
<span class="source-line-no">229</span><span id="line-229"> TraceUtil.trace(() -&gt; {</span>
<span class="source-line-no">230</span><span id="line-230"> if (!closed.compareAndSet(false, true)) {</span>
<span class="source-line-no">231</span><span id="line-231"> return;</span>
<span class="source-line-no">232</span><span id="line-232"> }</span>
<span class="source-line-no">233</span><span id="line-233"> LOG.info("Connection has been closed by {}.", Thread.currentThread().getName());</span>
<span class="source-line-no">234</span><span id="line-234"> if (LOG.isDebugEnabled()) {</span>
<span class="source-line-no">235</span><span id="line-235"> logCallStack(Thread.currentThread().getStackTrace());</span>
<span class="source-line-no">236</span><span id="line-236"> }</span>
<span class="source-line-no">237</span><span id="line-237"> IOUtils.closeQuietly(clusterStatusListener,</span>
<span class="source-line-no">238</span><span id="line-238"> e -&gt; LOG.warn("failed to close clusterStatusListener", e));</span>
<span class="source-line-no">239</span><span id="line-239"> IOUtils.closeQuietly(rpcClient, e -&gt; LOG.warn("failed to close rpcClient", e));</span>
<span class="source-line-no">240</span><span id="line-240"> IOUtils.closeQuietly(registry, e -&gt; LOG.warn("failed to close registry", e));</span>
<span class="source-line-no">241</span><span id="line-241"> synchronized (this) {</span>
<span class="source-line-no">242</span><span id="line-242"> if (choreService != null) {</span>
<span class="source-line-no">243</span><span id="line-243"> choreService.shutdown();</span>
<span class="source-line-no">244</span><span id="line-244"> choreService = null;</span>
<span class="source-line-no">245</span><span id="line-245"> }</span>
<span class="source-line-no">246</span><span id="line-246"> }</span>
<span class="source-line-no">247</span><span id="line-247"> if (metrics.isPresent()) {</span>
<span class="source-line-no">248</span><span id="line-248"> MetricsConnection.deleteMetricsConnection(metricsScope);</span>
<span class="source-line-no">249</span><span id="line-249"> }</span>
<span class="source-line-no">250</span><span id="line-250"> ConnectionOverAsyncConnection c = this.conn;</span>
<span class="source-line-no">251</span><span id="line-251"> if (c != null) {</span>
<span class="source-line-no">252</span><span id="line-252"> c.closePool();</span>
<span class="source-line-no">253</span><span id="line-253"> }</span>
<span class="source-line-no">254</span><span id="line-254"> }, "AsyncConnection.close");</span>
<span class="source-line-no">255</span><span id="line-255"> }</span>
<span class="source-line-no">256</span><span id="line-256"></span>
<span class="source-line-no">257</span><span id="line-257"> private void logCallStack(StackTraceElement[] stackTraceElements) {</span>
<span class="source-line-no">258</span><span id="line-258"> StringBuilder stackBuilder = new StringBuilder("Call stack:");</span>
<span class="source-line-no">259</span><span id="line-259"> for (StackTraceElement element : stackTraceElements) {</span>
<span class="source-line-no">260</span><span id="line-260"> stackBuilder.append("\n at ");</span>
<span class="source-line-no">261</span><span id="line-261"> stackBuilder.append(element);</span>
<span class="source-line-no">262</span><span id="line-262"> }</span>
<span class="source-line-no">263</span><span id="line-263"> stackBuilder.append("\n");</span>
<span class="source-line-no">264</span><span id="line-264"> LOG.debug(stackBuilder.toString());</span>
<span class="source-line-no">265</span><span id="line-265"> }</span>
<span class="source-line-no">266</span><span id="line-266"></span>
<span class="source-line-no">267</span><span id="line-267"> @Override</span>
<span class="source-line-no">268</span><span id="line-268"> public AsyncTableRegionLocator getRegionLocator(TableName tableName) {</span>
<span class="source-line-no">269</span><span id="line-269"> return new AsyncTableRegionLocatorImpl(tableName, this);</span>
<span class="source-line-no">270</span><span id="line-270"> }</span>
<span class="source-line-no">271</span><span id="line-271"></span>
<span class="source-line-no">272</span><span id="line-272"> @Override</span>
<span class="source-line-no">273</span><span id="line-273"> public void clearRegionLocationCache() {</span>
<span class="source-line-no">274</span><span id="line-274"> locator.clearCache();</span>
<span class="source-line-no">275</span><span id="line-275"> }</span>
<span class="source-line-no">276</span><span id="line-276"></span>
<span class="source-line-no">277</span><span id="line-277"> // we will override this method for testing retry caller, so do not remove this method.</span>
<span class="source-line-no">278</span><span id="line-278"> AsyncRegionLocator getLocator() {</span>
<span class="source-line-no">279</span><span id="line-279"> return locator;</span>
<span class="source-line-no">280</span><span id="line-280"> }</span>
<span class="source-line-no">281</span><span id="line-281"></span>
<span class="source-line-no">282</span><span id="line-282"> // ditto</span>
<span class="source-line-no">283</span><span id="line-283"> NonceGenerator getNonceGenerator() {</span>
<span class="source-line-no">284</span><span id="line-284"> return nonceGenerator;</span>
<span class="source-line-no">285</span><span id="line-285"> }</span>
<span class="source-line-no">286</span><span id="line-286"></span>
<span class="source-line-no">287</span><span id="line-287"> private ClientService.Interface createRegionServerStub(ServerName serverName) throws IOException {</span>
<span class="source-line-no">288</span><span id="line-288"> return ClientService.newStub(rpcClient.createRpcChannel(serverName, user, rpcTimeout));</span>
<span class="source-line-no">289</span><span id="line-289"> }</span>
<span class="source-line-no">290</span><span id="line-290"></span>
<span class="source-line-no">291</span><span id="line-291"> ClientService.Interface getRegionServerStub(ServerName serverName) throws IOException {</span>
<span class="source-line-no">292</span><span id="line-292"> return ConcurrentMapUtils.computeIfAbsentEx(rsStubs,</span>
<span class="source-line-no">293</span><span id="line-293"> getStubKey(ClientService.getDescriptor().getName(), serverName),</span>
<span class="source-line-no">294</span><span id="line-294"> () -&gt; createRegionServerStub(serverName));</span>
<span class="source-line-no">295</span><span id="line-295"> }</span>
<span class="source-line-no">296</span><span id="line-296"></span>
<span class="source-line-no">297</span><span id="line-297"> private MasterService.Interface createMasterStub(ServerName serverName) throws IOException {</span>
<span class="source-line-no">298</span><span id="line-298"> return MasterService.newStub(rpcClient.createRpcChannel(serverName, user, rpcTimeout));</span>
<span class="source-line-no">299</span><span id="line-299"> }</span>
<span class="source-line-no">300</span><span id="line-300"></span>
<span class="source-line-no">301</span><span id="line-301"> private AdminService.Interface createAdminServerStub(ServerName serverName) throws IOException {</span>
<span class="source-line-no">302</span><span id="line-302"> return AdminService.newStub(rpcClient.createRpcChannel(serverName, user, rpcTimeout));</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"> AdminService.Interface getAdminStub(ServerName serverName) throws IOException {</span>
<span class="source-line-no">306</span><span id="line-306"> return ConcurrentMapUtils.computeIfAbsentEx(adminStubs,</span>
<span class="source-line-no">307</span><span id="line-307"> getStubKey(AdminService.getDescriptor().getName(), serverName),</span>
<span class="source-line-no">308</span><span id="line-308"> () -&gt; createAdminServerStub(serverName));</span>
<span class="source-line-no">309</span><span id="line-309"> }</span>
<span class="source-line-no">310</span><span id="line-310"></span>
<span class="source-line-no">311</span><span id="line-311"> CompletableFuture&lt;MasterService.Interface&gt; getMasterStub() {</span>
<span class="source-line-no">312</span><span id="line-312"> return ConnectionUtils.getOrFetch(masterStub, masterStubMakeFuture, false, () -&gt; {</span>
<span class="source-line-no">313</span><span id="line-313"> CompletableFuture&lt;MasterService.Interface&gt; future = new CompletableFuture&lt;&gt;();</span>
<span class="source-line-no">314</span><span id="line-314"> addListener(registry.getActiveMaster(), (addr, error) -&gt; {</span>
<span class="source-line-no">315</span><span id="line-315"> if (error != null) {</span>
<span class="source-line-no">316</span><span id="line-316"> future.completeExceptionally(error);</span>
<span class="source-line-no">317</span><span id="line-317"> } else if (addr == null) {</span>
<span class="source-line-no">318</span><span id="line-318"> future.completeExceptionally(new MasterNotRunningException(</span>
<span class="source-line-no">319</span><span id="line-319"> "ZooKeeper available but no active master location found"));</span>
<span class="source-line-no">320</span><span id="line-320"> } else {</span>
<span class="source-line-no">321</span><span id="line-321"> LOG.debug("The fetched master address is {}", addr);</span>
<span class="source-line-no">322</span><span id="line-322"> try {</span>
<span class="source-line-no">323</span><span id="line-323"> future.complete(createMasterStub(addr));</span>
<span class="source-line-no">324</span><span id="line-324"> } catch (IOException e) {</span>
<span class="source-line-no">325</span><span id="line-325"> future.completeExceptionally(e);</span>
<span class="source-line-no">326</span><span id="line-326"> }</span>
<span class="source-line-no">327</span><span id="line-327"> }</span>
<span class="source-line-no">328</span><span id="line-328"></span>
<span class="source-line-no">329</span><span id="line-329"> });</span>
<span class="source-line-no">330</span><span id="line-330"> return future;</span>
<span class="source-line-no">331</span><span id="line-331"> }, stub -&gt; true, "master stub");</span>
<span class="source-line-no">332</span><span id="line-332"> }</span>
<span class="source-line-no">333</span><span id="line-333"></span>
<span class="source-line-no">334</span><span id="line-334"> String getClusterId() {</span>
<span class="source-line-no">335</span><span id="line-335"> try {</span>
<span class="source-line-no">336</span><span id="line-336"> return registry.getClusterId().get();</span>
<span class="source-line-no">337</span><span id="line-337"> } catch (InterruptedException | ExecutionException e) {</span>
<span class="source-line-no">338</span><span id="line-338"> LOG.error("Error fetching cluster ID: ", e);</span>
<span class="source-line-no">339</span><span id="line-339"> }</span>
<span class="source-line-no">340</span><span id="line-340"> return null;</span>
<span class="source-line-no">341</span><span id="line-341"> }</span>
<span class="source-line-no">342</span><span id="line-342"></span>
<span class="source-line-no">343</span><span id="line-343"> void clearMasterStubCache(MasterService.Interface stub) {</span>
<span class="source-line-no">344</span><span id="line-344"> masterStub.compareAndSet(stub, null);</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"> Optional&lt;ServerStatisticTracker&gt; getStatisticsTracker() {</span>
<span class="source-line-no">348</span><span id="line-348"> return stats;</span>
<span class="source-line-no">349</span><span id="line-349"> }</span>
<span class="source-line-no">350</span><span id="line-350"></span>
<span class="source-line-no">351</span><span id="line-351"> ClientBackoffPolicy getBackoffPolicy() {</span>
<span class="source-line-no">352</span><span id="line-352"> return backoffPolicy;</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"> @Override</span>
<span class="source-line-no">356</span><span id="line-356"> public AsyncTableBuilder&lt;AdvancedScanResultConsumer&gt; getTableBuilder(TableName tableName) {</span>
<span class="source-line-no">357</span><span id="line-357"> return new AsyncTableBuilderBase&lt;AdvancedScanResultConsumer&gt;(tableName, connConf) {</span>
<span class="source-line-no">358</span><span id="line-358"></span>
<span class="source-line-no">359</span><span id="line-359"> @Override</span>
<span class="source-line-no">360</span><span id="line-360"> public AsyncTable&lt;AdvancedScanResultConsumer&gt; build() {</span>
<span class="source-line-no">361</span><span id="line-361"> return new RawAsyncTableImpl(AsyncConnectionImpl.this, RETRY_TIMER, this);</span>
<span class="source-line-no">362</span><span id="line-362"> }</span>
<span class="source-line-no">363</span><span id="line-363"> };</span>
<span class="source-line-no">364</span><span id="line-364"> }</span>
<span class="source-line-no">365</span><span id="line-365"></span>
<span class="source-line-no">366</span><span id="line-366"> @Override</span>
<span class="source-line-no">367</span><span id="line-367"> public AsyncTableBuilder&lt;ScanResultConsumer&gt; getTableBuilder(TableName tableName,</span>
<span class="source-line-no">368</span><span id="line-368"> ExecutorService pool) {</span>
<span class="source-line-no">369</span><span id="line-369"> return new AsyncTableBuilderBase&lt;ScanResultConsumer&gt;(tableName, connConf) {</span>
<span class="source-line-no">370</span><span id="line-370"></span>
<span class="source-line-no">371</span><span id="line-371"> @Override</span>
<span class="source-line-no">372</span><span id="line-372"> public AsyncTable&lt;ScanResultConsumer&gt; build() {</span>
<span class="source-line-no">373</span><span id="line-373"> RawAsyncTableImpl rawTable =</span>
<span class="source-line-no">374</span><span id="line-374"> new RawAsyncTableImpl(AsyncConnectionImpl.this, RETRY_TIMER, this);</span>
<span class="source-line-no">375</span><span id="line-375"> return new AsyncTableImpl(rawTable, pool);</span>
<span class="source-line-no">376</span><span id="line-376"> }</span>
<span class="source-line-no">377</span><span id="line-377"> };</span>
<span class="source-line-no">378</span><span id="line-378"> }</span>
<span class="source-line-no">379</span><span id="line-379"></span>
<span class="source-line-no">380</span><span id="line-380"> @Override</span>
<span class="source-line-no">381</span><span id="line-381"> public AsyncAdminBuilder getAdminBuilder() {</span>
<span class="source-line-no">382</span><span id="line-382"> return new AsyncAdminBuilderBase(connConf) {</span>
<span class="source-line-no">383</span><span id="line-383"> @Override</span>
<span class="source-line-no">384</span><span id="line-384"> public AsyncAdmin build() {</span>
<span class="source-line-no">385</span><span id="line-385"> return new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, RETRY_TIMER, this);</span>
<span class="source-line-no">386</span><span id="line-386"> }</span>
<span class="source-line-no">387</span><span id="line-387"> };</span>
<span class="source-line-no">388</span><span id="line-388"> }</span>
<span class="source-line-no">389</span><span id="line-389"></span>
<span class="source-line-no">390</span><span id="line-390"> @Override</span>
<span class="source-line-no">391</span><span id="line-391"> public AsyncAdminBuilder getAdminBuilder(ExecutorService pool) {</span>
<span class="source-line-no">392</span><span id="line-392"> return new AsyncAdminBuilderBase(connConf) {</span>
<span class="source-line-no">393</span><span id="line-393"> @Override</span>
<span class="source-line-no">394</span><span id="line-394"> public AsyncAdmin build() {</span>
<span class="source-line-no">395</span><span id="line-395"> RawAsyncHBaseAdmin rawAdmin =</span>
<span class="source-line-no">396</span><span id="line-396"> new RawAsyncHBaseAdmin(AsyncConnectionImpl.this, RETRY_TIMER, this);</span>
<span class="source-line-no">397</span><span id="line-397"> return new AsyncHBaseAdmin(rawAdmin, pool);</span>
<span class="source-line-no">398</span><span id="line-398"> }</span>
<span class="source-line-no">399</span><span id="line-399"> };</span>
<span class="source-line-no">400</span><span id="line-400"> }</span>
<span class="source-line-no">401</span><span id="line-401"></span>
<span class="source-line-no">402</span><span id="line-402"> @Override</span>
<span class="source-line-no">403</span><span id="line-403"> public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName) {</span>
<span class="source-line-no">404</span><span id="line-404"> return new AsyncBufferedMutatorBuilderImpl(connConf, getTableBuilder(tableName), RETRY_TIMER);</span>
<span class="source-line-no">405</span><span id="line-405"> }</span>
<span class="source-line-no">406</span><span id="line-406"></span>
<span class="source-line-no">407</span><span id="line-407"> @Override</span>
<span class="source-line-no">408</span><span id="line-408"> public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName,</span>
<span class="source-line-no">409</span><span id="line-409"> ExecutorService pool) {</span>
<span class="source-line-no">410</span><span id="line-410"> return new AsyncBufferedMutatorBuilderImpl(connConf, getTableBuilder(tableName, pool),</span>
<span class="source-line-no">411</span><span id="line-411"> RETRY_TIMER);</span>
<span class="source-line-no">412</span><span id="line-412"> }</span>
<span class="source-line-no">413</span><span id="line-413"></span>
<span class="source-line-no">414</span><span id="line-414"> @Override</span>
<span class="source-line-no">415</span><span id="line-415"> public Connection toConnection() {</span>
<span class="source-line-no">416</span><span id="line-416"> ConnectionOverAsyncConnection c = this.conn;</span>
<span class="source-line-no">417</span><span id="line-417"> if (c != null) {</span>
<span class="source-line-no">418</span><span id="line-418"> return c;</span>
<span class="source-line-no">419</span><span id="line-419"> }</span>
<span class="source-line-no">420</span><span id="line-420"> synchronized (this) {</span>
<span class="source-line-no">421</span><span id="line-421"> c = this.conn;</span>
<span class="source-line-no">422</span><span id="line-422"> if (c != null) {</span>
<span class="source-line-no">423</span><span id="line-423"> return c;</span>
<span class="source-line-no">424</span><span id="line-424"> }</span>
<span class="source-line-no">425</span><span id="line-425"> c = new ConnectionOverAsyncConnection(this);</span>
<span class="source-line-no">426</span><span id="line-426"> this.conn = c;</span>
<span class="source-line-no">427</span><span id="line-427"> }</span>
<span class="source-line-no">428</span><span id="line-428"> return c;</span>
<span class="source-line-no">429</span><span id="line-429"> }</span>
<span class="source-line-no">430</span><span id="line-430"></span>
<span class="source-line-no">431</span><span id="line-431"> private Hbck getHbckInternal(ServerName masterServer) {</span>
<span class="source-line-no">432</span><span id="line-432"> Span.current().setAttribute(SERVER_NAME_KEY, masterServer.getServerName());</span>
<span class="source-line-no">433</span><span id="line-433"> // we will not create a new connection when creating a new protobuf stub, and for hbck there</span>
<span class="source-line-no">434</span><span id="line-434"> // will be no performance consideration, so for simplification we will create a new stub every</span>
<span class="source-line-no">435</span><span id="line-435"> // time instead of caching the stub here.</span>
<span class="source-line-no">436</span><span id="line-436"> return new HBaseHbck(MasterProtos.HbckService.newBlockingStub(</span>
<span class="source-line-no">437</span><span id="line-437"> rpcClient.createBlockingRpcChannel(masterServer, user, rpcTimeout)), rpcControllerFactory);</span>
<span class="source-line-no">438</span><span id="line-438"> }</span>
<span class="source-line-no">439</span><span id="line-439"></span>
<span class="source-line-no">440</span><span id="line-440"> @Override</span>
<span class="source-line-no">441</span><span id="line-441"> public CompletableFuture&lt;Hbck&gt; getHbck() {</span>
<span class="source-line-no">442</span><span id="line-442"> return TraceUtil.tracedFuture(() -&gt; {</span>
<span class="source-line-no">443</span><span id="line-443"> CompletableFuture&lt;Hbck&gt; future = new CompletableFuture&lt;&gt;();</span>
<span class="source-line-no">444</span><span id="line-444"> addListener(registry.getActiveMaster(), (sn, error) -&gt; {</span>
<span class="source-line-no">445</span><span id="line-445"> if (error != null) {</span>
<span class="source-line-no">446</span><span id="line-446"> future.completeExceptionally(error);</span>
<span class="source-line-no">447</span><span id="line-447"> } else {</span>
<span class="source-line-no">448</span><span id="line-448"> future.complete(getHbckInternal(sn));</span>
<span class="source-line-no">449</span><span id="line-449"> }</span>
<span class="source-line-no">450</span><span id="line-450"> });</span>
<span class="source-line-no">451</span><span id="line-451"> return future;</span>
<span class="source-line-no">452</span><span id="line-452"> }, "AsyncConnection.getHbck");</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"> @Override</span>
<span class="source-line-no">456</span><span id="line-456"> public Hbck getHbck(ServerName masterServer) {</span>
<span class="source-line-no">457</span><span id="line-457"> return TraceUtil.trace(() -&gt; getHbckInternal(masterServer), "AsyncConnection.getHbck");</span>
<span class="source-line-no">458</span><span id="line-458"> }</span>
<span class="source-line-no">459</span><span id="line-459"></span>
<span class="source-line-no">460</span><span id="line-460"> Optional&lt;MetricsConnection&gt; getConnectionMetrics() {</span>
<span class="source-line-no">461</span><span id="line-461"> return metrics;</span>
<span class="source-line-no">462</span><span id="line-462"> }</span>
<span class="source-line-no">463</span><span id="line-463">}</span>
</pre>
</div>
</main>
</body>
</html>