1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21 package org.apache.hadoop.hbase.master;
22
23
24 import io.netty.bootstrap.Bootstrap;
25 import io.netty.bootstrap.ChannelFactory;
26 import io.netty.buffer.Unpooled;
27 import io.netty.channel.Channel;
28 import io.netty.channel.ChannelException;
29 import io.netty.channel.ChannelHandlerContext;
30 import io.netty.channel.ChannelOption;
31 import io.netty.channel.EventLoopGroup;
32 import io.netty.channel.nio.NioEventLoopGroup;
33 import io.netty.channel.socket.DatagramChannel;
34 import io.netty.channel.socket.DatagramPacket;
35 import io.netty.channel.socket.InternetProtocolFamily;
36 import io.netty.channel.socket.nio.NioDatagramChannel;
37 import io.netty.handler.codec.MessageToMessageEncoder;
38 import org.apache.hadoop.hbase.classification.InterfaceAudience;
39 import io.netty.util.internal.StringUtil;
40 import org.apache.hadoop.conf.Configuration;
41 import org.apache.hadoop.hbase.Chore;
42 import org.apache.hadoop.hbase.ClusterStatus;
43 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
44 import org.apache.hadoop.hbase.HConstants;
45 import org.apache.hadoop.hbase.ServerName;
46 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
47 import org.apache.hadoop.hbase.util.Addressing;
48 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
49 import org.apache.hadoop.hbase.util.ExceptionUtil;
50 import org.apache.hadoop.hbase.util.Pair;
51 import org.apache.hadoop.hbase.util.ReflectionUtils;
52 import org.apache.hadoop.hbase.util.Threads;
53 import org.apache.hadoop.hbase.util.VersionInfo;
54
55 import java.io.Closeable;
56 import java.io.IOException;
57 import java.net.Inet6Address;
58 import java.net.InetAddress;
59 import java.net.InetSocketAddress;
60 import java.net.NetworkInterface;
61 import java.net.UnknownHostException;
62 import java.util.ArrayList;
63 import java.util.Collections;
64 import java.util.Comparator;
65 import java.util.List;
66 import java.util.Map;
67 import java.util.concurrent.ConcurrentHashMap;
68 import java.util.concurrent.ConcurrentMap;
69
70
71
72
73
74
75
76
77 @InterfaceAudience.Private
78 public class ClusterStatusPublisher extends Chore {
79
80
81
82
83
84 public static final String STATUS_PUBLISHER_CLASS = "hbase.status.publisher.class";
85 public static final Class<? extends ClusterStatusPublisher.Publisher>
86 DEFAULT_STATUS_PUBLISHER_CLASS =
87 org.apache.hadoop.hbase.master.ClusterStatusPublisher.MulticastPublisher.class;
88
89
90
91
92 public static final String STATUS_PUBLISH_PERIOD = "hbase.status.publish.period";
93 public static final int DEFAULT_STATUS_PUBLISH_PERIOD = 10000;
94
95 private long lastMessageTime = 0;
96 private final HMaster master;
97 private final int messagePeriod;
98 private final ConcurrentMap<ServerName, Integer> lastSent =
99 new ConcurrentHashMap<ServerName, Integer>();
100 private Publisher publisher;
101 private boolean connected = false;
102
103
104
105
106
107 public final static int MAX_SERVER_PER_MESSAGE = 10;
108
109
110
111
112
113 public final static int NB_SEND = 5;
114
115 public ClusterStatusPublisher(HMaster master, Configuration conf,
116 Class<? extends Publisher> publisherClass)
117 throws IOException {
118 super("HBase clusterStatusPublisher for " + master.getName(),
119 conf.getInt(STATUS_PUBLISH_PERIOD, DEFAULT_STATUS_PUBLISH_PERIOD), master);
120 this.master = master;
121 this.messagePeriod = conf.getInt(STATUS_PUBLISH_PERIOD, DEFAULT_STATUS_PUBLISH_PERIOD);
122 try {
123 this.publisher = publisherClass.newInstance();
124 } catch (InstantiationException e) {
125 throw new IOException("Can't create publisher " + publisherClass.getName(), e);
126 } catch (IllegalAccessException e) {
127 throw new IOException("Can't create publisher " + publisherClass.getName(), e);
128 }
129 this.publisher.connect(conf);
130 connected = true;
131 }
132
133
134 protected ClusterStatusPublisher() {
135 master = null;
136 messagePeriod = 0;
137 }
138
139 @Override
140 protected void chore() {
141 if (!connected) {
142 return;
143 }
144
145 List<ServerName> sns = generateDeadServersListToSend();
146 if (sns.isEmpty()) {
147
148 return;
149 }
150
151 final long curTime = EnvironmentEdgeManager.currentTime();
152 if (lastMessageTime > curTime - messagePeriod) {
153
154 return;
155 }
156
157
158 lastMessageTime = curTime;
159
160
161
162
163 ClusterStatus cs = new ClusterStatus(VersionInfo.getVersion(),
164 master.getMasterFileSystem().getClusterId().toString(),
165 null,
166 sns,
167 master.getServerName(),
168 null,
169 null,
170 null,
171 null);
172
173
174 publisher.publish(cs);
175 }
176
177 protected void cleanup() {
178 connected = false;
179 publisher.close();
180 }
181
182
183
184
185
186
187 protected List<ServerName> generateDeadServersListToSend() {
188
189 long since = EnvironmentEdgeManager.currentTime() - messagePeriod * 2;
190 for (Pair<ServerName, Long> dead : getDeadServers(since)) {
191 lastSent.putIfAbsent(dead.getFirst(), 0);
192 }
193
194
195 List<Map.Entry<ServerName, Integer>> entries = new ArrayList<Map.Entry<ServerName, Integer>>();
196 entries.addAll(lastSent.entrySet());
197 Collections.sort(entries, new Comparator<Map.Entry<ServerName, Integer>>() {
198 @Override
199 public int compare(Map.Entry<ServerName, Integer> o1, Map.Entry<ServerName, Integer> o2) {
200 return o1.getValue().compareTo(o2.getValue());
201 }
202 });
203
204
205 int max = entries.size() > MAX_SERVER_PER_MESSAGE ? MAX_SERVER_PER_MESSAGE : entries.size();
206 List<ServerName> res = new ArrayList<ServerName>(max);
207
208 for (int i = 0; i < max; i++) {
209 Map.Entry<ServerName, Integer> toSend = entries.get(i);
210 if (toSend.getValue() >= (NB_SEND - 1)) {
211 lastSent.remove(toSend.getKey());
212 } else {
213 lastSent.replace(toSend.getKey(), toSend.getValue(), toSend.getValue() + 1);
214 }
215
216 res.add(toSend.getKey());
217 }
218
219 return res;
220 }
221
222
223
224
225
226 protected List<Pair<ServerName, Long>> getDeadServers(long since) {
227 if (master.getServerManager() == null) {
228 return Collections.emptyList();
229 }
230
231 return master.getServerManager().getDeadServers().copyDeadServersSince(since);
232 }
233
234
235 public interface Publisher extends Closeable {
236
237 void connect(Configuration conf) throws IOException;
238
239 void publish(ClusterStatus cs);
240
241 @Override
242 void close();
243 }
244
245 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
246 public static class MulticastPublisher implements Publisher {
247 private DatagramChannel channel;
248 private final EventLoopGroup group = new NioEventLoopGroup(
249 1, Threads.newDaemonThreadFactory("hbase-master-clusterStatusPublisher"));
250
251 public MulticastPublisher() {
252 }
253
254 @Override
255 public void connect(Configuration conf) throws IOException {
256 String mcAddress = conf.get(HConstants.STATUS_MULTICAST_ADDRESS,
257 HConstants.DEFAULT_STATUS_MULTICAST_ADDRESS);
258 int port = conf.getInt(HConstants.STATUS_MULTICAST_PORT,
259 HConstants.DEFAULT_STATUS_MULTICAST_PORT);
260
261 final InetAddress ina;
262 try {
263 ina = InetAddress.getByName(mcAddress);
264 } catch (UnknownHostException e) {
265 close();
266 throw new IOException("Can't connect to " + mcAddress, e);
267 }
268
269 final InetSocketAddress isa = new InetSocketAddress(mcAddress, port);
270
271 InternetProtocolFamily family;
272 InetAddress localAddress;
273 if (ina instanceof Inet6Address) {
274 localAddress = Addressing.getIp6Address();
275 family = InternetProtocolFamily.IPv6;
276 }else{
277 localAddress = Addressing.getIp4Address();
278 family = InternetProtocolFamily.IPv4;
279 }
280 NetworkInterface ni = NetworkInterface.getByInetAddress(localAddress);
281
282 Bootstrap b = new Bootstrap();
283 b.group(group)
284 .channelFactory(new HBaseDatagramChannelFactory<Channel>(NioDatagramChannel.class, family))
285 .option(ChannelOption.SO_REUSEADDR, true)
286 .handler(new ClusterStatusEncoder(isa));
287
288 try {
289 channel = (DatagramChannel) b.bind(new InetSocketAddress(0)).sync().channel();
290 channel.joinGroup(ina, ni, null, channel.newPromise()).sync();
291 channel.connect(isa).sync();
292 } catch (InterruptedException e) {
293 close();
294 throw ExceptionUtil.asInterrupt(e);
295 }
296 }
297
298 private static final class HBaseDatagramChannelFactory<T extends Channel> implements ChannelFactory<T> {
299 private final Class<? extends T> clazz;
300 private InternetProtocolFamily family;
301
302 HBaseDatagramChannelFactory(Class<? extends T> clazz, InternetProtocolFamily family) {
303 this.clazz = clazz;
304 this.family = family;
305 }
306
307 @Override
308 public T newChannel() {
309 try {
310 return ReflectionUtils.instantiateWithCustomCtor(clazz.getName(),
311 new Class[] { InternetProtocolFamily.class }, new Object[] { family });
312
313 } catch (Throwable t) {
314 throw new ChannelException("Unable to create Channel from class " + clazz, t);
315 }
316 }
317
318 @Override
319 public String toString() {
320 return StringUtil.simpleClassName(clazz) + ".class";
321 }
322 }
323
324 private static class ClusterStatusEncoder extends MessageToMessageEncoder<ClusterStatus> {
325 final private InetSocketAddress isa;
326
327 private ClusterStatusEncoder(InetSocketAddress isa) {
328 this.isa = isa;
329 }
330
331 @Override
332 protected void encode(ChannelHandlerContext channelHandlerContext,
333 ClusterStatus clusterStatus, List<Object> objects) {
334 ClusterStatusProtos.ClusterStatus csp = clusterStatus.convert();
335 objects.add(new DatagramPacket(Unpooled.wrappedBuffer(csp.toByteArray()), isa));
336 }
337 }
338
339 @Override
340 public void publish(ClusterStatus cs) {
341 channel.writeAndFlush(cs).syncUninterruptibly();
342 }
343
344 @Override
345 public void close() {
346 if (channel != null) {
347 channel.close();
348 }
349 group.shutdownGracefully();
350 }
351 }
352 }