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