1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.zookeeper;
20
21 import java.io.BufferedReader;
22 import java.io.File;
23 import java.io.InterruptedIOException;
24 import java.io.IOException;
25 import java.io.InputStreamReader;
26 import java.io.OutputStream;
27 import java.io.Reader;
28 import java.net.BindException;
29 import java.net.InetSocketAddress;
30 import java.net.Socket;
31 import java.util.ArrayList;
32 import java.util.List;
33 import java.util.Random;
34
35 import org.apache.commons.logging.Log;
36 import org.apache.commons.logging.LogFactory;
37 import org.apache.hadoop.hbase.classification.InterfaceAudience;
38 import org.apache.hadoop.hbase.classification.InterfaceStability;
39 import org.apache.hadoop.conf.Configuration;
40 import org.apache.hadoop.hbase.HConstants;
41 import org.apache.zookeeper.server.NIOServerCnxnFactory;
42 import org.apache.zookeeper.server.ZooKeeperServer;
43 import org.apache.zookeeper.server.persistence.FileTxnLog;
44
45
46
47
48
49
50 @InterfaceAudience.Public
51 @InterfaceStability.Evolving
52 public class MiniZooKeeperCluster {
53 private static final Log LOG = LogFactory.getLog(MiniZooKeeperCluster.class);
54
55 private static final int TICK_TIME = 2000;
56 private static final int CONNECTION_TIMEOUT = 30000;
57
58 private boolean started;
59
60
61 private int defaultClientPort = 0;
62
63 private List<NIOServerCnxnFactory> standaloneServerFactoryList;
64 private List<ZooKeeperServer> zooKeeperServers;
65 private List<Integer> clientPortList;
66
67 private int activeZKServerIndex;
68 private int tickTime = 0;
69
70 private Configuration configuration;
71
72 public MiniZooKeeperCluster() {
73 this(new Configuration());
74 }
75
76 public MiniZooKeeperCluster(Configuration configuration) {
77 this.started = false;
78 this.configuration = configuration;
79 activeZKServerIndex = -1;
80 zooKeeperServers = new ArrayList<ZooKeeperServer>();
81 clientPortList = new ArrayList<Integer>();
82 standaloneServerFactoryList = new ArrayList<NIOServerCnxnFactory>();
83 }
84
85 public void setDefaultClientPort(int clientPort) {
86 if (clientPort <= 0) {
87 throw new IllegalArgumentException("Invalid default ZK client port: "
88 + clientPort);
89 }
90 this.defaultClientPort = clientPort;
91 }
92
93
94
95
96
97
98
99 private int selectClientPort() {
100 if (defaultClientPort > 0) {
101 return defaultClientPort;
102 }
103 return 0xc000 + new Random().nextInt(0x3f00);
104 }
105
106 public void setTickTime(int tickTime) {
107 this.tickTime = tickTime;
108 }
109
110 public int getBackupZooKeeperServerNum() {
111 return zooKeeperServers.size()-1;
112 }
113
114 public int getZooKeeperServerNum() {
115 return zooKeeperServers.size();
116 }
117
118
119 private static void setupTestEnv() {
120
121
122
123
124 System.setProperty("zookeeper.preAllocSize", "100");
125 FileTxnLog.setPreallocSize(100 * 1024);
126 }
127
128 public int startup(File baseDir) throws IOException, InterruptedException {
129 return startup(baseDir,1);
130 }
131
132
133
134
135
136
137
138
139
140 public int startup(File baseDir, int numZooKeeperServers) throws IOException,
141 InterruptedException {
142 if (numZooKeeperServers <= 0)
143 return -1;
144
145 setupTestEnv();
146 shutdown();
147
148 int tentativePort = selectClientPort();
149
150
151 for (int i = 0; i < numZooKeeperServers; i++) {
152 File dir = new File(baseDir, "zookeeper_"+i).getAbsoluteFile();
153 createDir(dir);
154 int tickTimeToUse;
155 if (this.tickTime > 0) {
156 tickTimeToUse = this.tickTime;
157 } else {
158 tickTimeToUse = TICK_TIME;
159 }
160 ZooKeeperServer server = new ZooKeeperServer(dir, dir, tickTimeToUse);
161 NIOServerCnxnFactory standaloneServerFactory;
162 while (true) {
163 try {
164 standaloneServerFactory = new NIOServerCnxnFactory();
165 standaloneServerFactory.configure(
166 new InetSocketAddress(tentativePort),
167 configuration.getInt(HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS, 1000));
168 } catch (BindException e) {
169 LOG.debug("Failed binding ZK Server to client port: " +
170 tentativePort, e);
171
172 if (defaultClientPort > 0) return -1;
173
174 tentativePort = selectClientPort();
175 continue;
176 }
177 break;
178 }
179
180
181 standaloneServerFactory.startup(server);
182
183 if (!waitForServerUp(tentativePort, CONNECTION_TIMEOUT)) {
184 throw new IOException("Waiting for startup of standalone server");
185 }
186
187
188 clientPortList.add(tentativePort);
189 standaloneServerFactoryList.add(standaloneServerFactory);
190 zooKeeperServers.add(server);
191 tentativePort++;
192 }
193
194
195 activeZKServerIndex = 0;
196 started = true;
197 int clientPort = clientPortList.get(activeZKServerIndex);
198 LOG.info("Started MiniZooKeeperCluster and ran successful 'stat' " +
199 "on client port=" + clientPort);
200 return clientPort;
201 }
202
203 private void createDir(File dir) throws IOException {
204 try {
205 if (!dir.exists()) {
206 dir.mkdirs();
207 }
208 } catch (SecurityException e) {
209 throw new IOException("creating dir: " + dir, e);
210 }
211 }
212
213
214
215
216 public void shutdown() throws IOException {
217 if (!started) {
218 return;
219 }
220
221
222 for (int i = 0; i < standaloneServerFactoryList.size(); i++) {
223 NIOServerCnxnFactory standaloneServerFactory =
224 standaloneServerFactoryList.get(i);
225 int clientPort = clientPortList.get(i);
226
227 standaloneServerFactory.shutdown();
228 if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) {
229 throw new IOException("Waiting for shutdown of standalone server");
230 }
231 }
232 for (ZooKeeperServer zkServer: zooKeeperServers) {
233
234 zkServer.getZKDatabase().close();
235 }
236
237
238 started = false;
239 activeZKServerIndex = 0;
240 standaloneServerFactoryList.clear();
241 clientPortList.clear();
242 zooKeeperServers.clear();
243
244 LOG.info("Shutdown MiniZK cluster with all ZK servers");
245 }
246
247
248
249
250
251
252 public int killCurrentActiveZooKeeperServer() throws IOException,
253 InterruptedException {
254 if (!started || activeZKServerIndex < 0 ) {
255 return -1;
256 }
257
258
259 NIOServerCnxnFactory standaloneServerFactory =
260 standaloneServerFactoryList.get(activeZKServerIndex);
261 int clientPort = clientPortList.get(activeZKServerIndex);
262
263 standaloneServerFactory.shutdown();
264 if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) {
265 throw new IOException("Waiting for shutdown of standalone server");
266 }
267
268 zooKeeperServers.get(activeZKServerIndex).getZKDatabase().close();
269
270
271 standaloneServerFactoryList.remove(activeZKServerIndex);
272 clientPortList.remove(activeZKServerIndex);
273 zooKeeperServers.remove(activeZKServerIndex);
274 LOG.info("Kill the current active ZK servers in the cluster " +
275 "on client port: " + clientPort);
276
277 if (standaloneServerFactoryList.size() == 0) {
278
279 return -1;
280 }
281 clientPort = clientPortList.get(activeZKServerIndex);
282 LOG.info("Activate a backup zk server in the cluster " +
283 "on client port: " + clientPort);
284
285 return clientPort;
286 }
287
288
289
290
291
292
293 public void killOneBackupZooKeeperServer() throws IOException,
294 InterruptedException {
295 if (!started || activeZKServerIndex < 0 ||
296 standaloneServerFactoryList.size() <= 1) {
297 return ;
298 }
299
300 int backupZKServerIndex = activeZKServerIndex+1;
301
302 NIOServerCnxnFactory standaloneServerFactory =
303 standaloneServerFactoryList.get(backupZKServerIndex);
304 int clientPort = clientPortList.get(backupZKServerIndex);
305
306 standaloneServerFactory.shutdown();
307 if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) {
308 throw new IOException("Waiting for shutdown of standalone server");
309 }
310
311 zooKeeperServers.get(backupZKServerIndex).getZKDatabase().close();
312
313
314 standaloneServerFactoryList.remove(backupZKServerIndex);
315 clientPortList.remove(backupZKServerIndex);
316 zooKeeperServers.remove(backupZKServerIndex);
317 LOG.info("Kill one backup ZK servers in the cluster " +
318 "on client port: " + clientPort);
319 }
320
321
322 private static boolean waitForServerDown(int port, long timeout) throws IOException {
323 long start = System.currentTimeMillis();
324 while (true) {
325 try {
326 Socket sock = new Socket("localhost", port);
327 try {
328 OutputStream outstream = sock.getOutputStream();
329 outstream.write("stat".getBytes());
330 outstream.flush();
331 } finally {
332 sock.close();
333 }
334 } catch (IOException e) {
335 return true;
336 }
337
338 if (System.currentTimeMillis() > start + timeout) {
339 break;
340 }
341 try {
342 Thread.sleep(250);
343 } catch (InterruptedException e) {
344 throw (InterruptedIOException)new InterruptedIOException().initCause(e);
345 }
346 }
347 return false;
348 }
349
350
351 private static boolean waitForServerUp(int port, long timeout) throws IOException {
352 long start = System.currentTimeMillis();
353 while (true) {
354 try {
355 Socket sock = new Socket("localhost", port);
356 BufferedReader reader = null;
357 try {
358 OutputStream outstream = sock.getOutputStream();
359 outstream.write("stat".getBytes());
360 outstream.flush();
361
362 Reader isr = new InputStreamReader(sock.getInputStream());
363 reader = new BufferedReader(isr);
364 String line = reader.readLine();
365 if (line != null && line.startsWith("Zookeeper version:")) {
366 return true;
367 }
368 } finally {
369 sock.close();
370 if (reader != null) {
371 reader.close();
372 }
373 }
374 } catch (IOException e) {
375
376 LOG.info("server localhost:" + port + " not up " + e);
377 }
378
379 if (System.currentTimeMillis() > start + timeout) {
380 break;
381 }
382 try {
383 Thread.sleep(250);
384 } catch (InterruptedException e) {
385 throw (InterruptedIOException)new InterruptedIOException().initCause(e);
386 }
387 }
388 return false;
389 }
390
391 public int getClientPort() {
392 return activeZKServerIndex < 0 || activeZKServerIndex >= clientPortList.size() ? -1
393 : clientPortList.get(activeZKServerIndex);
394 }
395 }