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