blob: f2e09ffe5ac478bdc74eefb65a18fb4961468b0e [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><a name="line.18"></a>
<span class="sourceLineNo">019</span><a name="line.19"></a>
<span class="sourceLineNo">020</span>package org.apache.hadoop.hbase.master;<a name="line.20"></a>
<span class="sourceLineNo">021</span><a name="line.21"></a>
<span class="sourceLineNo">022</span>import java.io.Closeable;<a name="line.22"></a>
<span class="sourceLineNo">023</span>import java.io.IOException;<a name="line.23"></a>
<span class="sourceLineNo">024</span>import java.net.Inet6Address;<a name="line.24"></a>
<span class="sourceLineNo">025</span>import java.net.InetAddress;<a name="line.25"></a>
<span class="sourceLineNo">026</span>import java.net.InetSocketAddress;<a name="line.26"></a>
<span class="sourceLineNo">027</span>import java.net.NetworkInterface;<a name="line.27"></a>
<span class="sourceLineNo">028</span>import java.net.UnknownHostException;<a name="line.28"></a>
<span class="sourceLineNo">029</span>import java.util.ArrayList;<a name="line.29"></a>
<span class="sourceLineNo">030</span>import java.util.Collections;<a name="line.30"></a>
<span class="sourceLineNo">031</span>import java.util.Comparator;<a name="line.31"></a>
<span class="sourceLineNo">032</span>import java.util.List;<a name="line.32"></a>
<span class="sourceLineNo">033</span>import java.util.Map;<a name="line.33"></a>
<span class="sourceLineNo">034</span>import java.util.concurrent.ConcurrentHashMap;<a name="line.34"></a>
<span class="sourceLineNo">035</span>import java.util.concurrent.ConcurrentMap;<a name="line.35"></a>
<span class="sourceLineNo">036</span>import org.apache.hadoop.conf.Configuration;<a name="line.36"></a>
<span class="sourceLineNo">037</span>import org.apache.hadoop.hbase.ClusterMetrics;<a name="line.37"></a>
<span class="sourceLineNo">038</span>import org.apache.hadoop.hbase.ClusterMetricsBuilder;<a name="line.38"></a>
<span class="sourceLineNo">039</span>import org.apache.hadoop.hbase.HBaseInterfaceAudience;<a name="line.39"></a>
<span class="sourceLineNo">040</span>import org.apache.hadoop.hbase.HConstants;<a name="line.40"></a>
<span class="sourceLineNo">041</span>import org.apache.hadoop.hbase.ScheduledChore;<a name="line.41"></a>
<span class="sourceLineNo">042</span>import org.apache.hadoop.hbase.ServerName;<a name="line.42"></a>
<span class="sourceLineNo">043</span>import org.apache.hadoop.hbase.util.Addressing;<a name="line.43"></a>
<span class="sourceLineNo">044</span>import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;<a name="line.44"></a>
<span class="sourceLineNo">045</span>import org.apache.hadoop.hbase.util.ExceptionUtil;<a name="line.45"></a>
<span class="sourceLineNo">046</span>import org.apache.hadoop.hbase.util.Pair;<a name="line.46"></a>
<span class="sourceLineNo">047</span>import org.apache.hadoop.hbase.util.ReflectionUtils;<a name="line.47"></a>
<span class="sourceLineNo">048</span>import org.apache.hadoop.hbase.util.Threads;<a name="line.48"></a>
<span class="sourceLineNo">049</span>import org.apache.hadoop.hbase.util.VersionInfo;<a name="line.49"></a>
<span class="sourceLineNo">050</span>import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;<a name="line.50"></a>
<span class="sourceLineNo">051</span>import org.apache.yetus.audience.InterfaceAudience;<a name="line.51"></a>
<span class="sourceLineNo">052</span>import org.apache.hbase.thirdparty.io.netty.bootstrap.Bootstrap;<a name="line.52"></a>
<span class="sourceLineNo">053</span>import org.apache.hbase.thirdparty.io.netty.buffer.Unpooled;<a name="line.53"></a>
<span class="sourceLineNo">054</span>import org.apache.hbase.thirdparty.io.netty.channel.Channel;<a name="line.54"></a>
<span class="sourceLineNo">055</span>import org.apache.hbase.thirdparty.io.netty.channel.ChannelException;<a name="line.55"></a>
<span class="sourceLineNo">056</span>import org.apache.hbase.thirdparty.io.netty.channel.ChannelFactory;<a name="line.56"></a>
<span class="sourceLineNo">057</span>import org.apache.hbase.thirdparty.io.netty.channel.ChannelHandlerContext;<a name="line.57"></a>
<span class="sourceLineNo">058</span>import org.apache.hbase.thirdparty.io.netty.channel.ChannelOption;<a name="line.58"></a>
<span class="sourceLineNo">059</span>import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;<a name="line.59"></a>
<span class="sourceLineNo">060</span>import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;<a name="line.60"></a>
<span class="sourceLineNo">061</span>import org.apache.hbase.thirdparty.io.netty.channel.socket.DatagramChannel;<a name="line.61"></a>
<span class="sourceLineNo">062</span>import org.apache.hbase.thirdparty.io.netty.channel.socket.DatagramPacket;<a name="line.62"></a>
<span class="sourceLineNo">063</span>import org.apache.hbase.thirdparty.io.netty.channel.socket.InternetProtocolFamily;<a name="line.63"></a>
<span class="sourceLineNo">064</span>import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioDatagramChannel;<a name="line.64"></a>
<span class="sourceLineNo">065</span>import org.apache.hbase.thirdparty.io.netty.handler.codec.MessageToMessageEncoder;<a name="line.65"></a>
<span class="sourceLineNo">066</span>import org.apache.hbase.thirdparty.io.netty.util.internal.StringUtil;<a name="line.66"></a>
<span class="sourceLineNo">067</span>import org.slf4j.Logger;<a name="line.67"></a>
<span class="sourceLineNo">068</span>import org.slf4j.LoggerFactory;<a name="line.68"></a>
<span class="sourceLineNo">069</span><a name="line.69"></a>
<span class="sourceLineNo">070</span><a name="line.70"></a>
<span class="sourceLineNo">071</span>/**<a name="line.71"></a>
<span class="sourceLineNo">072</span> * Class to publish the cluster status to the client. This allows them to know immediately<a name="line.72"></a>
<span class="sourceLineNo">073</span> * the dead region servers, hence to cut the connection they have with them, eventually stop<a name="line.73"></a>
<span class="sourceLineNo">074</span> * waiting on the socket. This improves the mean time to recover, and as well allows to increase<a name="line.74"></a>
<span class="sourceLineNo">075</span> * on the client the different timeouts, as the dead servers will be detected separately.<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 ClusterStatusPublisher extends ScheduledChore {<a name="line.78"></a>
<span class="sourceLineNo">079</span> private static Logger LOG = LoggerFactory.getLogger(ClusterStatusPublisher.class);<a name="line.79"></a>
<span class="sourceLineNo">080</span> /**<a name="line.80"></a>
<span class="sourceLineNo">081</span> * The implementation class used to publish the status. Default is null (no publish).<a name="line.81"></a>
<span class="sourceLineNo">082</span> * Use org.apache.hadoop.hbase.master.ClusterStatusPublisher.MulticastPublisher to multicast the<a name="line.82"></a>
<span class="sourceLineNo">083</span> * status.<a name="line.83"></a>
<span class="sourceLineNo">084</span> */<a name="line.84"></a>
<span class="sourceLineNo">085</span> public static final String STATUS_PUBLISHER_CLASS = "hbase.status.publisher.class";<a name="line.85"></a>
<span class="sourceLineNo">086</span> public static final Class&lt;? extends ClusterStatusPublisher.Publisher&gt;<a name="line.86"></a>
<span class="sourceLineNo">087</span> DEFAULT_STATUS_PUBLISHER_CLASS =<a name="line.87"></a>
<span class="sourceLineNo">088</span> org.apache.hadoop.hbase.master.ClusterStatusPublisher.MulticastPublisher.class;<a name="line.88"></a>
<span class="sourceLineNo">089</span><a name="line.89"></a>
<span class="sourceLineNo">090</span> /**<a name="line.90"></a>
<span class="sourceLineNo">091</span> * The minimum time between two status messages, in milliseconds.<a name="line.91"></a>
<span class="sourceLineNo">092</span> */<a name="line.92"></a>
<span class="sourceLineNo">093</span> public static final String STATUS_PUBLISH_PERIOD = "hbase.status.publish.period";<a name="line.93"></a>
<span class="sourceLineNo">094</span> public static final int DEFAULT_STATUS_PUBLISH_PERIOD = 10000;<a name="line.94"></a>
<span class="sourceLineNo">095</span><a name="line.95"></a>
<span class="sourceLineNo">096</span> private long lastMessageTime = 0;<a name="line.96"></a>
<span class="sourceLineNo">097</span> private final HMaster master;<a name="line.97"></a>
<span class="sourceLineNo">098</span> private final int messagePeriod; // time between two message<a name="line.98"></a>
<span class="sourceLineNo">099</span> private final ConcurrentMap&lt;ServerName, Integer&gt; lastSent = new ConcurrentHashMap&lt;&gt;();<a name="line.99"></a>
<span class="sourceLineNo">100</span> private Publisher publisher;<a name="line.100"></a>
<span class="sourceLineNo">101</span> private boolean connected = false;<a name="line.101"></a>
<span class="sourceLineNo">102</span><a name="line.102"></a>
<span class="sourceLineNo">103</span> /**<a name="line.103"></a>
<span class="sourceLineNo">104</span> * We want to limit the size of the protobuf message sent, do fit into a single packet.<a name="line.104"></a>
<span class="sourceLineNo">105</span> * a reasonable size for ip / ethernet is less than 1Kb.<a name="line.105"></a>
<span class="sourceLineNo">106</span> */<a name="line.106"></a>
<span class="sourceLineNo">107</span> public final static int MAX_SERVER_PER_MESSAGE = 10;<a name="line.107"></a>
<span class="sourceLineNo">108</span><a name="line.108"></a>
<span class="sourceLineNo">109</span> /**<a name="line.109"></a>
<span class="sourceLineNo">110</span> * If a server dies, we're sending the information multiple times in case a receiver misses the<a name="line.110"></a>
<span class="sourceLineNo">111</span> * message.<a name="line.111"></a>
<span class="sourceLineNo">112</span> */<a name="line.112"></a>
<span class="sourceLineNo">113</span> public final static int NB_SEND = 5;<a name="line.113"></a>
<span class="sourceLineNo">114</span><a name="line.114"></a>
<span class="sourceLineNo">115</span> public ClusterStatusPublisher(HMaster master, Configuration conf,<a name="line.115"></a>
<span class="sourceLineNo">116</span> Class&lt;? extends Publisher&gt; publisherClass)<a name="line.116"></a>
<span class="sourceLineNo">117</span> throws IOException {<a name="line.117"></a>
<span class="sourceLineNo">118</span> super("ClusterStatusPublisher for=" + master.getName(), master, conf.getInt(<a name="line.118"></a>
<span class="sourceLineNo">119</span> STATUS_PUBLISH_PERIOD, DEFAULT_STATUS_PUBLISH_PERIOD));<a name="line.119"></a>
<span class="sourceLineNo">120</span> this.master = master;<a name="line.120"></a>
<span class="sourceLineNo">121</span> this.messagePeriod = conf.getInt(STATUS_PUBLISH_PERIOD, DEFAULT_STATUS_PUBLISH_PERIOD);<a name="line.121"></a>
<span class="sourceLineNo">122</span> try {<a name="line.122"></a>
<span class="sourceLineNo">123</span> this.publisher = publisherClass.getDeclaredConstructor().newInstance();<a name="line.123"></a>
<span class="sourceLineNo">124</span> } catch (Exception e) {<a name="line.124"></a>
<span class="sourceLineNo">125</span> throw new IOException("Can't create publisher " + publisherClass.getName(), e);<a name="line.125"></a>
<span class="sourceLineNo">126</span> }<a name="line.126"></a>
<span class="sourceLineNo">127</span> this.publisher.connect(conf);<a name="line.127"></a>
<span class="sourceLineNo">128</span> connected = true;<a name="line.128"></a>
<span class="sourceLineNo">129</span> }<a name="line.129"></a>
<span class="sourceLineNo">130</span><a name="line.130"></a>
<span class="sourceLineNo">131</span> @Override<a name="line.131"></a>
<span class="sourceLineNo">132</span> public String toString() {<a name="line.132"></a>
<span class="sourceLineNo">133</span> return super.toString() + ", publisher=" + this.publisher + ", connected=" + this.connected;<a name="line.133"></a>
<span class="sourceLineNo">134</span> }<a name="line.134"></a>
<span class="sourceLineNo">135</span><a name="line.135"></a>
<span class="sourceLineNo">136</span> // For tests only<a name="line.136"></a>
<span class="sourceLineNo">137</span> protected ClusterStatusPublisher() {<a name="line.137"></a>
<span class="sourceLineNo">138</span> master = null;<a name="line.138"></a>
<span class="sourceLineNo">139</span> messagePeriod = 0;<a name="line.139"></a>
<span class="sourceLineNo">140</span> }<a name="line.140"></a>
<span class="sourceLineNo">141</span><a name="line.141"></a>
<span class="sourceLineNo">142</span> @Override<a name="line.142"></a>
<span class="sourceLineNo">143</span> protected void chore() {<a name="line.143"></a>
<span class="sourceLineNo">144</span> if (!isConnected()) {<a name="line.144"></a>
<span class="sourceLineNo">145</span> return;<a name="line.145"></a>
<span class="sourceLineNo">146</span> }<a name="line.146"></a>
<span class="sourceLineNo">147</span><a name="line.147"></a>
<span class="sourceLineNo">148</span> List&lt;ServerName&gt; sns = generateDeadServersListToSend();<a name="line.148"></a>
<span class="sourceLineNo">149</span> if (sns.isEmpty()) {<a name="line.149"></a>
<span class="sourceLineNo">150</span> // Nothing to send. Done.<a name="line.150"></a>
<span class="sourceLineNo">151</span> return;<a name="line.151"></a>
<span class="sourceLineNo">152</span> }<a name="line.152"></a>
<span class="sourceLineNo">153</span><a name="line.153"></a>
<span class="sourceLineNo">154</span> final long curTime = EnvironmentEdgeManager.currentTime();<a name="line.154"></a>
<span class="sourceLineNo">155</span> if (lastMessageTime &gt; curTime - messagePeriod) {<a name="line.155"></a>
<span class="sourceLineNo">156</span> // We already sent something less than 10 second ago. Done.<a name="line.156"></a>
<span class="sourceLineNo">157</span> return;<a name="line.157"></a>
<span class="sourceLineNo">158</span> }<a name="line.158"></a>
<span class="sourceLineNo">159</span><a name="line.159"></a>
<span class="sourceLineNo">160</span> // Ok, we're going to send something then.<a name="line.160"></a>
<span class="sourceLineNo">161</span> lastMessageTime = curTime;<a name="line.161"></a>
<span class="sourceLineNo">162</span><a name="line.162"></a>
<span class="sourceLineNo">163</span> // We're reusing an existing protobuf message, but we don't send everything.<a name="line.163"></a>
<span class="sourceLineNo">164</span> // This could be extended in the future, for example if we want to send stuff like the<a name="line.164"></a>
<span class="sourceLineNo">165</span> // hbase:meta server name.<a name="line.165"></a>
<span class="sourceLineNo">166</span> publisher.publish(ClusterMetricsBuilder.newBuilder()<a name="line.166"></a>
<span class="sourceLineNo">167</span> .setHBaseVersion(VersionInfo.getVersion())<a name="line.167"></a>
<span class="sourceLineNo">168</span> .setClusterId(master.getMasterFileSystem().getClusterId().toString())<a name="line.168"></a>
<span class="sourceLineNo">169</span> .setMasterName(master.getServerName())<a name="line.169"></a>
<span class="sourceLineNo">170</span> .setDeadServerNames(sns)<a name="line.170"></a>
<span class="sourceLineNo">171</span> .build());<a name="line.171"></a>
<span class="sourceLineNo">172</span> }<a name="line.172"></a>
<span class="sourceLineNo">173</span><a name="line.173"></a>
<span class="sourceLineNo">174</span> @Override<a name="line.174"></a>
<span class="sourceLineNo">175</span> protected synchronized void cleanup() {<a name="line.175"></a>
<span class="sourceLineNo">176</span> connected = false;<a name="line.176"></a>
<span class="sourceLineNo">177</span> publisher.close();<a name="line.177"></a>
<span class="sourceLineNo">178</span> }<a name="line.178"></a>
<span class="sourceLineNo">179</span><a name="line.179"></a>
<span class="sourceLineNo">180</span> private synchronized boolean isConnected() {<a name="line.180"></a>
<span class="sourceLineNo">181</span> return this.connected;<a name="line.181"></a>
<span class="sourceLineNo">182</span> }<a name="line.182"></a>
<span class="sourceLineNo">183</span><a name="line.183"></a>
<span class="sourceLineNo">184</span> /**<a name="line.184"></a>
<span class="sourceLineNo">185</span> * Create the dead server to send. A dead server is sent NB_SEND times. We send at max<a name="line.185"></a>
<span class="sourceLineNo">186</span> * MAX_SERVER_PER_MESSAGE at a time. if there are too many dead servers, we send the newly<a name="line.186"></a>
<span class="sourceLineNo">187</span> * dead first.<a name="line.187"></a>
<span class="sourceLineNo">188</span> */<a name="line.188"></a>
<span class="sourceLineNo">189</span> protected List&lt;ServerName&gt; generateDeadServersListToSend() {<a name="line.189"></a>
<span class="sourceLineNo">190</span> // We're getting the message sent since last time, and add them to the list<a name="line.190"></a>
<span class="sourceLineNo">191</span> long since = EnvironmentEdgeManager.currentTime() - messagePeriod * 2;<a name="line.191"></a>
<span class="sourceLineNo">192</span> for (Pair&lt;ServerName, Long&gt; dead : getDeadServers(since)) {<a name="line.192"></a>
<span class="sourceLineNo">193</span> lastSent.putIfAbsent(dead.getFirst(), 0);<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> // We're sending the new deads first.<a name="line.196"></a>
<span class="sourceLineNo">197</span> List&lt;Map.Entry&lt;ServerName, Integer&gt;&gt; entries = new ArrayList&lt;&gt;(lastSent.entrySet());<a name="line.197"></a>
<span class="sourceLineNo">198</span> Collections.sort(entries, new Comparator&lt;Map.Entry&lt;ServerName, Integer&gt;&gt;() {<a name="line.198"></a>
<span class="sourceLineNo">199</span> @Override<a name="line.199"></a>
<span class="sourceLineNo">200</span> public int compare(Map.Entry&lt;ServerName, Integer&gt; o1, Map.Entry&lt;ServerName, Integer&gt; o2) {<a name="line.200"></a>
<span class="sourceLineNo">201</span> return o1.getValue().compareTo(o2.getValue());<a name="line.201"></a>
<span class="sourceLineNo">202</span> }<a name="line.202"></a>
<span class="sourceLineNo">203</span> });<a name="line.203"></a>
<span class="sourceLineNo">204</span><a name="line.204"></a>
<span class="sourceLineNo">205</span> // With a limit of MAX_SERVER_PER_MESSAGE<a name="line.205"></a>
<span class="sourceLineNo">206</span> int max = entries.size() &gt; MAX_SERVER_PER_MESSAGE ? MAX_SERVER_PER_MESSAGE : entries.size();<a name="line.206"></a>
<span class="sourceLineNo">207</span> List&lt;ServerName&gt; res = new ArrayList&lt;&gt;(max);<a name="line.207"></a>
<span class="sourceLineNo">208</span><a name="line.208"></a>
<span class="sourceLineNo">209</span> for (int i = 0; i &lt; max; i++) {<a name="line.209"></a>
<span class="sourceLineNo">210</span> Map.Entry&lt;ServerName, Integer&gt; toSend = entries.get(i);<a name="line.210"></a>
<span class="sourceLineNo">211</span> if (toSend.getValue() &gt;= (NB_SEND - 1)) {<a name="line.211"></a>
<span class="sourceLineNo">212</span> lastSent.remove(toSend.getKey());<a name="line.212"></a>
<span class="sourceLineNo">213</span> } else {<a name="line.213"></a>
<span class="sourceLineNo">214</span> lastSent.replace(toSend.getKey(), toSend.getValue(), toSend.getValue() + 1);<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> res.add(toSend.getKey());<a name="line.217"></a>
<span class="sourceLineNo">218</span> }<a name="line.218"></a>
<span class="sourceLineNo">219</span><a name="line.219"></a>
<span class="sourceLineNo">220</span> return res;<a name="line.220"></a>
<span class="sourceLineNo">221</span> }<a name="line.221"></a>
<span class="sourceLineNo">222</span><a name="line.222"></a>
<span class="sourceLineNo">223</span> /**<a name="line.223"></a>
<span class="sourceLineNo">224</span> * Get the servers which died since a given timestamp.<a name="line.224"></a>
<span class="sourceLineNo">225</span> * protected because it can be subclassed by the tests.<a name="line.225"></a>
<span class="sourceLineNo">226</span> */<a name="line.226"></a>
<span class="sourceLineNo">227</span> protected List&lt;Pair&lt;ServerName, Long&gt;&gt; getDeadServers(long since) {<a name="line.227"></a>
<span class="sourceLineNo">228</span> if (master.getServerManager() == null) {<a name="line.228"></a>
<span class="sourceLineNo">229</span> return Collections.emptyList();<a name="line.229"></a>
<span class="sourceLineNo">230</span> }<a name="line.230"></a>
<span class="sourceLineNo">231</span><a name="line.231"></a>
<span class="sourceLineNo">232</span> return master.getServerManager().getDeadServers().copyDeadServersSince(since);<a name="line.232"></a>
<span class="sourceLineNo">233</span> }<a name="line.233"></a>
<span class="sourceLineNo">234</span><a name="line.234"></a>
<span class="sourceLineNo">235</span><a name="line.235"></a>
<span class="sourceLineNo">236</span> public interface Publisher extends Closeable {<a name="line.236"></a>
<span class="sourceLineNo">237</span><a name="line.237"></a>
<span class="sourceLineNo">238</span> void connect(Configuration conf) throws IOException;<a name="line.238"></a>
<span class="sourceLineNo">239</span><a name="line.239"></a>
<span class="sourceLineNo">240</span> void publish(ClusterMetrics cs);<a name="line.240"></a>
<span class="sourceLineNo">241</span><a name="line.241"></a>
<span class="sourceLineNo">242</span> @Override<a name="line.242"></a>
<span class="sourceLineNo">243</span> void close();<a name="line.243"></a>
<span class="sourceLineNo">244</span> }<a name="line.244"></a>
<span class="sourceLineNo">245</span><a name="line.245"></a>
<span class="sourceLineNo">246</span> @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)<a name="line.246"></a>
<span class="sourceLineNo">247</span> public static class MulticastPublisher implements Publisher {<a name="line.247"></a>
<span class="sourceLineNo">248</span> private DatagramChannel channel;<a name="line.248"></a>
<span class="sourceLineNo">249</span> private final EventLoopGroup group = new NioEventLoopGroup(1,<a name="line.249"></a>
<span class="sourceLineNo">250</span> new ThreadFactoryBuilder().setNameFormat("hbase-master-clusterStatusPublisher-pool-%d")<a name="line.250"></a>
<span class="sourceLineNo">251</span> .setDaemon(true).setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build());<a name="line.251"></a>
<span class="sourceLineNo">252</span><a name="line.252"></a>
<span class="sourceLineNo">253</span> public MulticastPublisher() {<a name="line.253"></a>
<span class="sourceLineNo">254</span> }<a name="line.254"></a>
<span class="sourceLineNo">255</span><a name="line.255"></a>
<span class="sourceLineNo">256</span> @Override<a name="line.256"></a>
<span class="sourceLineNo">257</span> public String toString() {<a name="line.257"></a>
<span class="sourceLineNo">258</span> return "channel=" + this.channel;<a name="line.258"></a>
<span class="sourceLineNo">259</span> }<a name="line.259"></a>
<span class="sourceLineNo">260</span><a name="line.260"></a>
<span class="sourceLineNo">261</span> @Override<a name="line.261"></a>
<span class="sourceLineNo">262</span> public void connect(Configuration conf) throws IOException {<a name="line.262"></a>
<span class="sourceLineNo">263</span> String mcAddress = conf.get(HConstants.STATUS_MULTICAST_ADDRESS,<a name="line.263"></a>
<span class="sourceLineNo">264</span> HConstants.DEFAULT_STATUS_MULTICAST_ADDRESS);<a name="line.264"></a>
<span class="sourceLineNo">265</span> int port = conf.getInt(HConstants.STATUS_MULTICAST_PORT,<a name="line.265"></a>
<span class="sourceLineNo">266</span> HConstants.DEFAULT_STATUS_MULTICAST_PORT);<a name="line.266"></a>
<span class="sourceLineNo">267</span> String bindAddress = conf.get(HConstants.STATUS_MULTICAST_PUBLISHER_BIND_ADDRESS,<a name="line.267"></a>
<span class="sourceLineNo">268</span> HConstants.DEFAULT_STATUS_MULTICAST_PUBLISHER_BIND_ADDRESS);<a name="line.268"></a>
<span class="sourceLineNo">269</span> String niName = conf.get(HConstants.STATUS_MULTICAST_NI_NAME);<a name="line.269"></a>
<span class="sourceLineNo">270</span><a name="line.270"></a>
<span class="sourceLineNo">271</span> final InetAddress ina;<a name="line.271"></a>
<span class="sourceLineNo">272</span> try {<a name="line.272"></a>
<span class="sourceLineNo">273</span> ina = InetAddress.getByName(mcAddress);<a name="line.273"></a>
<span class="sourceLineNo">274</span> } catch (UnknownHostException e) {<a name="line.274"></a>
<span class="sourceLineNo">275</span> close();<a name="line.275"></a>
<span class="sourceLineNo">276</span> throw new IOException("Can't connect to " + mcAddress, e);<a name="line.276"></a>
<span class="sourceLineNo">277</span> }<a name="line.277"></a>
<span class="sourceLineNo">278</span> final InetSocketAddress isa = new InetSocketAddress(mcAddress, port);<a name="line.278"></a>
<span class="sourceLineNo">279</span><a name="line.279"></a>
<span class="sourceLineNo">280</span> InternetProtocolFamily family;<a name="line.280"></a>
<span class="sourceLineNo">281</span> NetworkInterface ni;<a name="line.281"></a>
<span class="sourceLineNo">282</span> if (niName != null) {<a name="line.282"></a>
<span class="sourceLineNo">283</span> if (ina instanceof Inet6Address) {<a name="line.283"></a>
<span class="sourceLineNo">284</span> family = InternetProtocolFamily.IPv6;<a name="line.284"></a>
<span class="sourceLineNo">285</span> } else {<a name="line.285"></a>
<span class="sourceLineNo">286</span> family = InternetProtocolFamily.IPv4;<a name="line.286"></a>
<span class="sourceLineNo">287</span> }<a name="line.287"></a>
<span class="sourceLineNo">288</span> ni = NetworkInterface.getByName(niName);<a name="line.288"></a>
<span class="sourceLineNo">289</span> } else {<a name="line.289"></a>
<span class="sourceLineNo">290</span> InetAddress localAddress;<a name="line.290"></a>
<span class="sourceLineNo">291</span> if (ina instanceof Inet6Address) {<a name="line.291"></a>
<span class="sourceLineNo">292</span> localAddress = Addressing.getIp6Address();<a name="line.292"></a>
<span class="sourceLineNo">293</span> family = InternetProtocolFamily.IPv6;<a name="line.293"></a>
<span class="sourceLineNo">294</span> } else {<a name="line.294"></a>
<span class="sourceLineNo">295</span> localAddress = Addressing.getIp4Address();<a name="line.295"></a>
<span class="sourceLineNo">296</span> family = InternetProtocolFamily.IPv4;<a name="line.296"></a>
<span class="sourceLineNo">297</span> }<a name="line.297"></a>
<span class="sourceLineNo">298</span> ni = NetworkInterface.getByInetAddress(localAddress);<a name="line.298"></a>
<span class="sourceLineNo">299</span> }<a name="line.299"></a>
<span class="sourceLineNo">300</span> Bootstrap b = new Bootstrap();<a name="line.300"></a>
<span class="sourceLineNo">301</span> b.group(group)<a name="line.301"></a>
<span class="sourceLineNo">302</span> .channelFactory(new HBaseDatagramChannelFactory&lt;Channel&gt;(NioDatagramChannel.class, family))<a name="line.302"></a>
<span class="sourceLineNo">303</span> .option(ChannelOption.SO_REUSEADDR, true)<a name="line.303"></a>
<span class="sourceLineNo">304</span> .handler(new ClusterMetricsEncoder(isa));<a name="line.304"></a>
<span class="sourceLineNo">305</span> try {<a name="line.305"></a>
<span class="sourceLineNo">306</span> LOG.debug("Channel bindAddress={}, networkInterface={}, INA={}", bindAddress, ni, ina);<a name="line.306"></a>
<span class="sourceLineNo">307</span> channel = (DatagramChannel) b.bind(bindAddress, 0).sync().channel();<a name="line.307"></a>
<span class="sourceLineNo">308</span> channel.joinGroup(ina, ni, null, channel.newPromise()).sync();<a name="line.308"></a>
<span class="sourceLineNo">309</span> channel.connect(isa).sync();<a name="line.309"></a>
<span class="sourceLineNo">310</span> // Set into configuration in case many networks available. Do this for tests so that<a name="line.310"></a>
<span class="sourceLineNo">311</span> // server and client use same Interface (presuming share same Configuration).<a name="line.311"></a>
<span class="sourceLineNo">312</span> // TestAsyncTableRSCrashPublish was failing when connected to VPN because extra networks<a name="line.312"></a>
<span class="sourceLineNo">313</span> // available with Master binding on one Interface and client on another so test failed.<a name="line.313"></a>
<span class="sourceLineNo">314</span> if (ni != null) {<a name="line.314"></a>
<span class="sourceLineNo">315</span> conf.set(HConstants.STATUS_MULTICAST_NI_NAME, ni.getName());<a name="line.315"></a>
<span class="sourceLineNo">316</span> }<a name="line.316"></a>
<span class="sourceLineNo">317</span> } catch (InterruptedException e) {<a name="line.317"></a>
<span class="sourceLineNo">318</span> close();<a name="line.318"></a>
<span class="sourceLineNo">319</span> throw ExceptionUtil.asInterrupt(e);<a name="line.319"></a>
<span class="sourceLineNo">320</span> }<a name="line.320"></a>
<span class="sourceLineNo">321</span> }<a name="line.321"></a>
<span class="sourceLineNo">322</span><a name="line.322"></a>
<span class="sourceLineNo">323</span> private static final class HBaseDatagramChannelFactory&lt;T extends Channel&gt;<a name="line.323"></a>
<span class="sourceLineNo">324</span> implements ChannelFactory&lt;T&gt; {<a name="line.324"></a>
<span class="sourceLineNo">325</span> private final Class&lt;? extends T&gt; clazz;<a name="line.325"></a>
<span class="sourceLineNo">326</span> private final InternetProtocolFamily family;<a name="line.326"></a>
<span class="sourceLineNo">327</span><a name="line.327"></a>
<span class="sourceLineNo">328</span> HBaseDatagramChannelFactory(Class&lt;? extends T&gt; clazz, InternetProtocolFamily family) {<a name="line.328"></a>
<span class="sourceLineNo">329</span> this.clazz = clazz;<a name="line.329"></a>
<span class="sourceLineNo">330</span> this.family = family;<a name="line.330"></a>
<span class="sourceLineNo">331</span> }<a name="line.331"></a>
<span class="sourceLineNo">332</span><a name="line.332"></a>
<span class="sourceLineNo">333</span> @Override<a name="line.333"></a>
<span class="sourceLineNo">334</span> public T newChannel() {<a name="line.334"></a>
<span class="sourceLineNo">335</span> try {<a name="line.335"></a>
<span class="sourceLineNo">336</span> return ReflectionUtils.instantiateWithCustomCtor(clazz.getName(),<a name="line.336"></a>
<span class="sourceLineNo">337</span> new Class[] { InternetProtocolFamily.class }, new Object[] { family });<a name="line.337"></a>
<span class="sourceLineNo">338</span><a name="line.338"></a>
<span class="sourceLineNo">339</span> } catch (Throwable t) {<a name="line.339"></a>
<span class="sourceLineNo">340</span> throw new ChannelException("Unable to create Channel from class " + clazz, t);<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><a name="line.343"></a>
<span class="sourceLineNo">344</span> @Override<a name="line.344"></a>
<span class="sourceLineNo">345</span> public String toString() {<a name="line.345"></a>
<span class="sourceLineNo">346</span> return StringUtil.simpleClassName(clazz) + ".class";<a name="line.346"></a>
<span class="sourceLineNo">347</span> }<a name="line.347"></a>
<span class="sourceLineNo">348</span> }<a name="line.348"></a>
<span class="sourceLineNo">349</span><a name="line.349"></a>
<span class="sourceLineNo">350</span> private static final class ClusterMetricsEncoder<a name="line.350"></a>
<span class="sourceLineNo">351</span> extends MessageToMessageEncoder&lt;ClusterMetrics&gt; {<a name="line.351"></a>
<span class="sourceLineNo">352</span> final private InetSocketAddress isa;<a name="line.352"></a>
<span class="sourceLineNo">353</span><a name="line.353"></a>
<span class="sourceLineNo">354</span> private ClusterMetricsEncoder(InetSocketAddress isa) {<a name="line.354"></a>
<span class="sourceLineNo">355</span> this.isa = isa;<a name="line.355"></a>
<span class="sourceLineNo">356</span> }<a name="line.356"></a>
<span class="sourceLineNo">357</span><a name="line.357"></a>
<span class="sourceLineNo">358</span> @Override<a name="line.358"></a>
<span class="sourceLineNo">359</span> protected void encode(ChannelHandlerContext channelHandlerContext,<a name="line.359"></a>
<span class="sourceLineNo">360</span> ClusterMetrics clusterStatus, List&lt;Object&gt; objects) {<a name="line.360"></a>
<span class="sourceLineNo">361</span> objects.add(new DatagramPacket(Unpooled.wrappedBuffer(<a name="line.361"></a>
<span class="sourceLineNo">362</span> ClusterMetricsBuilder.toClusterStatus(clusterStatus).toByteArray()), isa));<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 void publish(ClusterMetrics cs) {<a name="line.367"></a>
<span class="sourceLineNo">368</span> LOG.info("PUBLISH {}", cs);<a name="line.368"></a>
<span class="sourceLineNo">369</span> channel.writeAndFlush(cs).syncUninterruptibly();<a name="line.369"></a>
<span class="sourceLineNo">370</span> }<a name="line.370"></a>
<span class="sourceLineNo">371</span><a name="line.371"></a>
<span class="sourceLineNo">372</span> @Override<a name="line.372"></a>
<span class="sourceLineNo">373</span> public void close() {<a name="line.373"></a>
<span class="sourceLineNo">374</span> if (channel != null) {<a name="line.374"></a>
<span class="sourceLineNo">375</span> channel.close();<a name="line.375"></a>
<span class="sourceLineNo">376</span> }<a name="line.376"></a>
<span class="sourceLineNo">377</span> group.shutdownGracefully();<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>}<a name="line.380"></a>
</pre>
</div>
</body>
</html>