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 org.apache.hadoop.classification.InterfaceAudience;
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.hbase.Chore;
27 import org.apache.hadoop.hbase.ClusterStatus;
28 import org.apache.hadoop.hbase.HConstants;
29 import org.apache.hadoop.hbase.ServerName;
30 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
31 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
32 import org.apache.hadoop.hbase.util.Pair;
33 import org.apache.hadoop.hbase.util.Threads;
34 import org.apache.hadoop.hbase.util.VersionInfo;
35 import org.jboss.netty.bootstrap.ConnectionlessBootstrap;
36 import org.jboss.netty.channel.ChannelEvent;
37 import org.jboss.netty.channel.ChannelHandlerContext;
38 import org.jboss.netty.channel.ChannelUpstreamHandler;
39 import org.jboss.netty.channel.Channels;
40 import org.jboss.netty.channel.socket.DatagramChannel;
41 import org.jboss.netty.channel.socket.DatagramChannelFactory;
42 import org.jboss.netty.channel.socket.oio.OioDatagramChannelFactory;
43 import org.jboss.netty.handler.codec.protobuf.ProtobufEncoder;
44
45 import java.io.Closeable;
46 import java.io.IOException;
47 import java.net.InetAddress;
48 import java.net.InetSocketAddress;
49 import java.net.UnknownHostException;
50 import java.util.ArrayList;
51 import java.util.Collections;
52 import java.util.Comparator;
53 import java.util.List;
54 import java.util.Map;
55 import java.util.concurrent.ConcurrentHashMap;
56 import java.util.concurrent.ConcurrentMap;
57 import java.util.concurrent.ExecutorService;
58 import java.util.concurrent.Executors;
59
60
61
62
63
64
65
66 @InterfaceAudience.Private
67 public class ClusterStatusPublisher extends Chore {
68
69
70
71
72
73 public static final String STATUS_PUBLISHER_CLASS = "hbase.status.publisher.class";
74 public static final Class<? extends ClusterStatusPublisher.Publisher>
75 DEFAULT_STATUS_PUBLISHER_CLASS = null;
76
77
78
79
80 public static final String STATUS_PUBLISH_PERIOD = "hbase.status.publish.period";
81 public static final int DEFAULT_STATUS_PUBLISH_PERIOD = 10000;
82
83 private long lastMessageTime = 0;
84 private final HMaster master;
85 private final int messagePeriod;
86 private final ConcurrentMap<ServerName, Integer> lastSent =
87 new ConcurrentHashMap<ServerName, Integer>();
88 private Publisher publisher;
89 private boolean connected = false;
90
91
92
93
94
95 public static int MAX_SERVER_PER_MESSAGE = 10;
96
97
98
99
100
101 public static int NB_SEND = 5;
102
103 public ClusterStatusPublisher(HMaster master, Configuration conf,
104 Class<? extends Publisher> publisherClass)
105 throws IOException {
106 super("HBase clusterStatusPublisher for " + master.getName(),
107 conf.getInt(STATUS_PUBLISH_PERIOD, DEFAULT_STATUS_PUBLISH_PERIOD), master);
108 this.master = master;
109 this.messagePeriod = conf.getInt(STATUS_PUBLISH_PERIOD, DEFAULT_STATUS_PUBLISH_PERIOD);
110 try {
111 this.publisher = publisherClass.newInstance();
112 } catch (InstantiationException e) {
113 throw new IOException("Can't create publisher " + publisherClass.getName(), e);
114 } catch (IllegalAccessException e) {
115 throw new IOException("Can't create publisher " + publisherClass.getName(), e);
116 }
117 this.publisher.connect(conf);
118 connected = true;
119 }
120
121
122 protected ClusterStatusPublisher() {
123 master = null;
124 messagePeriod = 0;
125 }
126
127 @Override
128 protected void chore() {
129 if (!connected) {
130 return;
131 }
132
133 List<ServerName> sns = generateDeadServersListToSend();
134 if (sns.isEmpty()) {
135
136 return;
137 }
138
139 final long curTime = EnvironmentEdgeManager.currentTimeMillis();
140 if (lastMessageTime > curTime - messagePeriod) {
141
142 return;
143 }
144
145
146 lastMessageTime = curTime;
147
148
149
150
151 ClusterStatus cs = new ClusterStatus(VersionInfo.getVersion(),
152 master.getMasterFileSystem().getClusterId().toString(),
153 null,
154 sns,
155 master.getServerName(),
156 null,
157 null,
158 null,
159 null);
160
161
162 publisher.publish(cs);
163 }
164
165 protected void cleanup() {
166 connected = false;
167 publisher.close();
168 }
169
170
171
172
173
174
175 protected List<ServerName> generateDeadServersListToSend() {
176
177 long since = EnvironmentEdgeManager.currentTimeMillis() - messagePeriod * 2;
178 for (Pair<ServerName, Long> dead : getDeadServers(since)) {
179 lastSent.putIfAbsent(dead.getFirst(), 0);
180 }
181
182
183 List<Map.Entry<ServerName, Integer>> entries = new ArrayList<Map.Entry<ServerName, Integer>>();
184 entries.addAll(lastSent.entrySet());
185 Collections.sort(entries, new Comparator<Map.Entry<ServerName, Integer>>() {
186 @Override
187 public int compare(Map.Entry<ServerName, Integer> o1, Map.Entry<ServerName, Integer> o2) {
188 return o1.getValue().compareTo(o2.getValue());
189 }
190 });
191
192
193 int max = entries.size() > MAX_SERVER_PER_MESSAGE ? MAX_SERVER_PER_MESSAGE : entries.size();
194 List<ServerName> res = new ArrayList<ServerName>(max);
195
196 for (int i = 0; i < max; i++) {
197 Map.Entry<ServerName, Integer> toSend = entries.get(i);
198 if (toSend.getValue() >= (NB_SEND - 1)) {
199 lastSent.remove(toSend.getKey());
200 } else {
201 lastSent.replace(toSend.getKey(), toSend.getValue(), toSend.getValue() + 1);
202 }
203
204 res.add(toSend.getKey());
205 }
206
207 return res;
208 }
209
210
211
212
213
214 protected List<Pair<ServerName, Long>> getDeadServers(long since) {
215 if (master.getServerManager() == null) {
216 return Collections.emptyList();
217 }
218
219 return master.getServerManager().getDeadServers().copyDeadServersSince(since);
220 }
221
222
223 public interface Publisher extends Closeable {
224
225 void connect(Configuration conf) throws IOException;
226
227 void publish(ClusterStatus cs);
228
229 @Override
230 void close();
231 }
232
233 public static class MulticastPublisher implements Publisher {
234 private DatagramChannel channel;
235 private final ExecutorService service = Executors.newSingleThreadExecutor(
236 Threads.newDaemonThreadFactory("hbase-master-clusterStatus-worker"));
237
238 public MulticastPublisher() {
239 }
240
241 @Override
242 public void connect(Configuration conf) throws IOException {
243 String mcAddress = conf.get(HConstants.STATUS_MULTICAST_ADDRESS,
244 HConstants.DEFAULT_STATUS_MULTICAST_ADDRESS);
245 int port = conf.getInt(HConstants.STATUS_MULTICAST_PORT,
246 HConstants.DEFAULT_STATUS_MULTICAST_PORT);
247
248
249 DatagramChannelFactory f = new OioDatagramChannelFactory(service);
250
251 ConnectionlessBootstrap b = new ConnectionlessBootstrap(f);
252 b.setPipeline(Channels.pipeline(new ProtobufEncoder(),
253 new ChannelUpstreamHandler() {
254 @Override
255 public void handleUpstream(ChannelHandlerContext ctx, ChannelEvent e)
256 throws Exception {
257
258 }
259 }));
260
261
262 channel = (DatagramChannel) b.bind(new InetSocketAddress(0));
263 channel.getConfig().setReuseAddress(true);
264
265 InetAddress ina;
266 try {
267 ina = InetAddress.getByName(mcAddress);
268 } catch (UnknownHostException e) {
269 throw new IOException("Can't connect to " + mcAddress, e);
270 }
271 channel.joinGroup(ina);
272 channel.connect(new InetSocketAddress(mcAddress, port));
273 }
274
275 @Override
276 public void publish(ClusterStatus cs) {
277 ClusterStatusProtos.ClusterStatus csp = cs.convert();
278 channel.write(csp);
279 }
280
281 @Override
282 public void close() {
283 if (channel != null) {
284 channel.close();
285 }
286 service.shutdown();
287 }
288 }
289 }