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