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