blob: b3013a1adf597c13feef21b2d8b5550a9ab99ab6 [file] [log] [blame]
<!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 java.util.ArrayList;<a name="line.20"></a>
<span class="sourceLineNo">021</span>import java.util.Collections;<a name="line.21"></a>
<span class="sourceLineNo">022</span>import java.util.List;<a name="line.22"></a>
<span class="sourceLineNo">023</span>import java.util.Set;<a name="line.23"></a>
<span class="sourceLineNo">024</span>import java.util.concurrent.CompletableFuture;<a name="line.24"></a>
<span class="sourceLineNo">025</span>import org.apache.hadoop.conf.Configuration;<a name="line.25"></a>
<span class="sourceLineNo">026</span>import org.apache.hadoop.hbase.ServerName;<a name="line.26"></a>
<span class="sourceLineNo">027</span>import org.apache.hadoop.hbase.ipc.HBaseRpcController;<a name="line.27"></a>
<span class="sourceLineNo">028</span>import org.apache.hadoop.hbase.ipc.RpcClient;<a name="line.28"></a>
<span class="sourceLineNo">029</span>import org.apache.hadoop.hbase.ipc.RpcClientFactory;<a name="line.29"></a>
<span class="sourceLineNo">030</span>import org.apache.hadoop.hbase.ipc.RpcControllerFactory;<a name="line.30"></a>
<span class="sourceLineNo">031</span>import org.apache.hadoop.hbase.security.User;<a name="line.31"></a>
<span class="sourceLineNo">032</span>import org.apache.hadoop.hbase.util.FutureUtils;<a name="line.32"></a>
<span class="sourceLineNo">033</span>import org.apache.yetus.audience.InterfaceAudience;<a name="line.33"></a>
<span class="sourceLineNo">034</span>import org.slf4j.Logger;<a name="line.34"></a>
<span class="sourceLineNo">035</span>import org.slf4j.LoggerFactory;<a name="line.35"></a>
<span class="sourceLineNo">036</span><a name="line.36"></a>
<span class="sourceLineNo">037</span>import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;<a name="line.37"></a>
<span class="sourceLineNo">038</span>import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;<a name="line.38"></a>
<span class="sourceLineNo">039</span><a name="line.39"></a>
<span class="sourceLineNo">040</span>import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.ConnectionRegistryService;<a name="line.40"></a>
<span class="sourceLineNo">041</span>import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetConnectionRegistryRequest;<a name="line.41"></a>
<span class="sourceLineNo">042</span>import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetConnectionRegistryResponse;<a name="line.42"></a>
<span class="sourceLineNo">043</span><a name="line.43"></a>
<span class="sourceLineNo">044</span>/**<a name="line.44"></a>
<span class="sourceLineNo">045</span> * Fetch cluster id through special preamble header.<a name="line.45"></a>
<span class="sourceLineNo">046</span> * &lt;p&gt;<a name="line.46"></a>
<span class="sourceLineNo">047</span> * An instance of this class should only be used once, like:<a name="line.47"></a>
<span class="sourceLineNo">048</span> *<a name="line.48"></a>
<span class="sourceLineNo">049</span> * &lt;pre&gt;<a name="line.49"></a>
<span class="sourceLineNo">050</span> * new ClusterIdFetcher().fetchClusterId()<a name="line.50"></a>
<span class="sourceLineNo">051</span> * &lt;/pre&gt;<a name="line.51"></a>
<span class="sourceLineNo">052</span> *<a name="line.52"></a>
<span class="sourceLineNo">053</span> * Calling the fetchClusterId multiple times will lead unexpected behavior.<a name="line.53"></a>
<span class="sourceLineNo">054</span> * &lt;p&gt;<a name="line.54"></a>
<span class="sourceLineNo">055</span> * See HBASE-25051 for more details.<a name="line.55"></a>
<span class="sourceLineNo">056</span> */<a name="line.56"></a>
<span class="sourceLineNo">057</span>@InterfaceAudience.Private<a name="line.57"></a>
<span class="sourceLineNo">058</span>class ClusterIdFetcher {<a name="line.58"></a>
<span class="sourceLineNo">059</span><a name="line.59"></a>
<span class="sourceLineNo">060</span> private static final Logger LOG = LoggerFactory.getLogger(ClusterIdFetcher.class);<a name="line.60"></a>
<span class="sourceLineNo">061</span><a name="line.61"></a>
<span class="sourceLineNo">062</span> private final List&lt;ServerName&gt; bootstrapServers;<a name="line.62"></a>
<span class="sourceLineNo">063</span><a name="line.63"></a>
<span class="sourceLineNo">064</span> private final User user;<a name="line.64"></a>
<span class="sourceLineNo">065</span><a name="line.65"></a>
<span class="sourceLineNo">066</span> private final RpcClient rpcClient;<a name="line.66"></a>
<span class="sourceLineNo">067</span><a name="line.67"></a>
<span class="sourceLineNo">068</span> private final RpcControllerFactory rpcControllerFactory;<a name="line.68"></a>
<span class="sourceLineNo">069</span><a name="line.69"></a>
<span class="sourceLineNo">070</span> private final CompletableFuture&lt;String&gt; future;<a name="line.70"></a>
<span class="sourceLineNo">071</span><a name="line.71"></a>
<span class="sourceLineNo">072</span> ClusterIdFetcher(Configuration conf, User user, RpcControllerFactory rpcControllerFactory,<a name="line.72"></a>
<span class="sourceLineNo">073</span> Set&lt;ServerName&gt; bootstrapServers) {<a name="line.73"></a>
<span class="sourceLineNo">074</span> this.user = user;<a name="line.74"></a>
<span class="sourceLineNo">075</span> // use null cluster id here as we do not know the cluster id yet, we will fetch it through this<a name="line.75"></a>
<span class="sourceLineNo">076</span> // rpc client<a name="line.76"></a>
<span class="sourceLineNo">077</span> this.rpcClient = RpcClientFactory.createClient(conf, null);<a name="line.77"></a>
<span class="sourceLineNo">078</span> this.rpcControllerFactory = rpcControllerFactory;<a name="line.78"></a>
<span class="sourceLineNo">079</span> this.bootstrapServers = new ArrayList&lt;ServerName&gt;(bootstrapServers);<a name="line.79"></a>
<span class="sourceLineNo">080</span> // shuffle the bootstrap servers so we will not always fetch from the same one<a name="line.80"></a>
<span class="sourceLineNo">081</span> Collections.shuffle(this.bootstrapServers);<a name="line.81"></a>
<span class="sourceLineNo">082</span> future = new CompletableFuture&lt;String&gt;();<a name="line.82"></a>
<span class="sourceLineNo">083</span> }<a name="line.83"></a>
<span class="sourceLineNo">084</span><a name="line.84"></a>
<span class="sourceLineNo">085</span> /**<a name="line.85"></a>
<span class="sourceLineNo">086</span> * Try get cluster id from the server with the given {@code index} in {@link #bootstrapServers}.<a name="line.86"></a>
<span class="sourceLineNo">087</span> */<a name="line.87"></a>
<span class="sourceLineNo">088</span> private void getClusterId(int index) {<a name="line.88"></a>
<span class="sourceLineNo">089</span> ServerName server = bootstrapServers.get(index);<a name="line.89"></a>
<span class="sourceLineNo">090</span> LOG.debug("Going to request {} for getting cluster id", server);<a name="line.90"></a>
<span class="sourceLineNo">091</span> // user and rpcTimeout are both not important here, as we will not actually send any rpc calls<a name="line.91"></a>
<span class="sourceLineNo">092</span> // out, only a preamble connection header, but if we pass null as user, there will be NPE in<a name="line.92"></a>
<span class="sourceLineNo">093</span> // some code paths...<a name="line.93"></a>
<span class="sourceLineNo">094</span> RpcChannel channel = rpcClient.createRpcChannel(server, user, 0);<a name="line.94"></a>
<span class="sourceLineNo">095</span> ConnectionRegistryService.Interface stub = ConnectionRegistryService.newStub(channel);<a name="line.95"></a>
<span class="sourceLineNo">096</span> HBaseRpcController controller = rpcControllerFactory.newController();<a name="line.96"></a>
<span class="sourceLineNo">097</span> stub.getConnectionRegistry(controller, GetConnectionRegistryRequest.getDefaultInstance(),<a name="line.97"></a>
<span class="sourceLineNo">098</span> new RpcCallback&lt;GetConnectionRegistryResponse&gt;() {<a name="line.98"></a>
<span class="sourceLineNo">099</span><a name="line.99"></a>
<span class="sourceLineNo">100</span> @Override<a name="line.100"></a>
<span class="sourceLineNo">101</span> public void run(GetConnectionRegistryResponse resp) {<a name="line.101"></a>
<span class="sourceLineNo">102</span> if (!controller.failed()) {<a name="line.102"></a>
<span class="sourceLineNo">103</span> LOG.debug("Got connection registry info: {}", resp);<a name="line.103"></a>
<span class="sourceLineNo">104</span> future.complete(resp.getClusterId());<a name="line.104"></a>
<span class="sourceLineNo">105</span> return;<a name="line.105"></a>
<span class="sourceLineNo">106</span> }<a name="line.106"></a>
<span class="sourceLineNo">107</span> if (ConnectionUtils.isUnexpectedPreambleHeaderException(controller.getFailed())) {<a name="line.107"></a>
<span class="sourceLineNo">108</span> // this means we have connected to an old server where it does not support passing<a name="line.108"></a>
<span class="sourceLineNo">109</span> // cluster id through preamble connnection header, so we fallback to use null<a name="line.109"></a>
<span class="sourceLineNo">110</span> // cluster id, which is the old behavior<a name="line.110"></a>
<span class="sourceLineNo">111</span> LOG.debug("Failed to get connection registry info, should be an old server,"<a name="line.111"></a>
<span class="sourceLineNo">112</span> + " fallback to use null cluster id", controller.getFailed());<a name="line.112"></a>
<span class="sourceLineNo">113</span> future.complete(null);<a name="line.113"></a>
<span class="sourceLineNo">114</span> } else {<a name="line.114"></a>
<span class="sourceLineNo">115</span> LOG.debug("Failed to get connection registry info", controller.getFailed());<a name="line.115"></a>
<span class="sourceLineNo">116</span> if (index == bootstrapServers.size() - 1) {<a name="line.116"></a>
<span class="sourceLineNo">117</span> future.completeExceptionally(controller.getFailed());<a name="line.117"></a>
<span class="sourceLineNo">118</span> } else {<a name="line.118"></a>
<span class="sourceLineNo">119</span> // try next bootstrap server<a name="line.119"></a>
<span class="sourceLineNo">120</span> getClusterId(index + 1);<a name="line.120"></a>
<span class="sourceLineNo">121</span> }<a name="line.121"></a>
<span class="sourceLineNo">122</span> }<a name="line.122"></a>
<span class="sourceLineNo">123</span> }<a name="line.123"></a>
<span class="sourceLineNo">124</span> });<a name="line.124"></a>
<span class="sourceLineNo">125</span><a name="line.125"></a>
<span class="sourceLineNo">126</span> }<a name="line.126"></a>
<span class="sourceLineNo">127</span><a name="line.127"></a>
<span class="sourceLineNo">128</span> CompletableFuture&lt;String&gt; fetchClusterId() {<a name="line.128"></a>
<span class="sourceLineNo">129</span> getClusterId(0);<a name="line.129"></a>
<span class="sourceLineNo">130</span> // close the rpc client after we finish the request<a name="line.130"></a>
<span class="sourceLineNo">131</span> FutureUtils.addListener(future, (r, e) -&gt; rpcClient.close());<a name="line.131"></a>
<span class="sourceLineNo">132</span> return future;<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>
</pre>
</div>
</body>
</html>