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