1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.client;
21
22
23 import org.apache.commons.logging.Log;
24 import org.apache.commons.logging.LogFactory;
25 import org.apache.hadoop.classification.InterfaceAudience;
26 import org.apache.hadoop.conf.Configuration;
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.Threads;
32 import org.jboss.netty.bootstrap.ConnectionlessBootstrap;
33 import org.jboss.netty.channel.ChannelHandlerContext;
34 import org.jboss.netty.channel.Channels;
35 import org.jboss.netty.channel.ExceptionEvent;
36 import org.jboss.netty.channel.MessageEvent;
37 import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
38 import org.jboss.netty.channel.socket.DatagramChannel;
39 import org.jboss.netty.channel.socket.DatagramChannelFactory;
40 import org.jboss.netty.channel.socket.oio.OioDatagramChannelFactory;
41 import org.jboss.netty.handler.codec.protobuf.ProtobufDecoder;
42
43 import java.io.Closeable;
44 import java.io.IOException;
45 import java.lang.reflect.Constructor;
46 import java.lang.reflect.InvocationTargetException;
47 import java.net.InetAddress;
48 import java.net.InetSocketAddress;
49 import java.net.UnknownHostException;
50 import java.util.ArrayList;
51 import java.util.List;
52 import java.util.concurrent.ExecutorService;
53 import java.util.concurrent.Executors;
54
55
56
57
58
59
60
61 @InterfaceAudience.Private
62 class ClusterStatusListener implements Closeable {
63 private static final Log LOG = LogFactory.getLog(ClusterStatusListener.class);
64 private final List<ServerName> deadServers = new ArrayList<ServerName>();
65 private final DeadServerHandler deadServerHandler;
66 private final Listener listener;
67
68
69
70
71 public static final String STATUS_LISTENER_CLASS = "hbase.status.listener.class";
72 public static final Class<? extends Listener> DEFAULT_STATUS_LISTENER_CLASS = null;
73
74
75
76
77 public interface DeadServerHandler {
78
79
80
81
82
83
84
85 public void newDead(ServerName sn);
86 }
87
88
89
90
91
92 static interface Listener extends Closeable {
93
94
95
96 @Override
97 public void close();
98
99
100
101
102
103
104
105 public void connect(Configuration conf) throws IOException;
106 }
107
108 public ClusterStatusListener(DeadServerHandler dsh, Configuration conf,
109 Class<? extends Listener> listenerClass) throws IOException {
110 this.deadServerHandler = dsh;
111 try {
112 Constructor<? extends Listener> ctor =
113 listenerClass.getConstructor(ClusterStatusListener.class);
114 this.listener = ctor.newInstance(this);
115 } catch (InstantiationException e) {
116 throw new IOException("Can't create listener " + listenerClass.getName(), e);
117 } catch (IllegalAccessException e) {
118 throw new IOException("Can't create listener " + listenerClass.getName(), e);
119 } catch (NoSuchMethodException e) {
120 throw new IllegalStateException();
121 } catch (InvocationTargetException e) {
122 throw new IllegalStateException();
123 }
124
125 this.listener.connect(conf);
126 }
127
128
129
130
131
132
133 public void receive(ClusterStatus ncs) {
134 if (ncs.getDeadServerNames() != null) {
135 for (ServerName sn : ncs.getDeadServerNames()) {
136 if (!isDeadServer(sn)) {
137 LOG.info("There is a new dead server: " + sn);
138 deadServers.add(sn);
139 if (deadServerHandler != null) {
140 deadServerHandler.newDead(sn);
141 }
142 }
143 }
144 }
145 }
146
147 @Override
148 public void close() {
149 listener.close();
150 }
151
152
153
154
155
156
157
158 public boolean isDeadServer(ServerName sn) {
159 if (sn.getStartcode() <= 0) {
160 return false;
161 }
162
163 for (ServerName dead : deadServers) {
164 if (dead.getStartcode() >= sn.getStartcode() &&
165 dead.getPort() == sn.getPort() &&
166 dead.getHostname().equals(sn.getHostname())) {
167 return true;
168 }
169 }
170
171 return false;
172 }
173
174
175
176
177
178 class MultiCastListener implements Listener {
179 private DatagramChannel channel;
180 private final ExecutorService service = Executors.newSingleThreadExecutor(
181 Threads.newDaemonThreadFactory("hbase-client-clusterStatus-multiCastListener"));
182
183
184 public MultiCastListener() {
185 }
186
187 public void connect(Configuration conf) throws IOException {
188
189 DatagramChannelFactory f = new OioDatagramChannelFactory(service);
190
191 ConnectionlessBootstrap b = new ConnectionlessBootstrap(f);
192 b.setPipeline(Channels.pipeline(
193 new ProtobufDecoder(ClusterStatusProtos.ClusterStatus.getDefaultInstance()),
194 new ClusterStatusHandler()));
195
196 String mcAddress = conf.get(HConstants.STATUS_MULTICAST_ADDRESS,
197 HConstants.DEFAULT_STATUS_MULTICAST_ADDRESS);
198 int port = conf.getInt(HConstants.STATUS_MULTICAST_PORT,
199 HConstants.DEFAULT_STATUS_MULTICAST_PORT);
200 channel = (DatagramChannel) b.bind(new InetSocketAddress(mcAddress, port));
201
202 channel.getConfig().setReuseAddress(true);
203
204 InetAddress ina;
205 try {
206 ina = InetAddress.getByName(mcAddress);
207 } catch (UnknownHostException e) {
208 throw new IOException("Can't connect to " + mcAddress, e);
209 }
210 channel.joinGroup(ina);
211 }
212
213 @Override
214 public void close() {
215 if (channel != null) {
216 channel.close();
217 channel = null;
218 }
219 service.shutdown();
220 }
221
222
223
224
225
226 private class ClusterStatusHandler extends SimpleChannelUpstreamHandler {
227
228 @Override
229 public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception {
230 ClusterStatusProtos.ClusterStatus csp = (ClusterStatusProtos.ClusterStatus) e.getMessage();
231 ClusterStatus ncs = ClusterStatus.convert(csp);
232 receive(ncs);
233 }
234
235
236
237
238
239 @Override
240 public void exceptionCaught(
241 ChannelHandlerContext ctx, ExceptionEvent e) throws Exception {
242 LOG.error("Unexpected exception, continuing.", e.getCause());
243 }
244 }
245 }
246 }