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.classification.InterfaceAudience;
37 import org.apache.hadoop.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 public int startup(File baseDir, int numZooKeeperServers) throws IOException,
142 InterruptedException {
143 if (numZooKeeperServers <= 0)
144 return -1;
145
146 setupTestEnv();
147 shutdown();
148
149 int tentativePort = selectClientPort();
150
151
152 for (int i = 0; i < numZooKeeperServers; i++) {
153 File dir = new File(baseDir, "zookeeper_"+i).getAbsoluteFile();
154 recreateDir(dir);
155 int tickTimeToUse;
156 if (this.tickTime > 0) {
157 tickTimeToUse = this.tickTime;
158 } else {
159 tickTimeToUse = TICK_TIME;
160 }
161 ZooKeeperServer server = new ZooKeeperServer(dir, dir, tickTimeToUse);
162 NIOServerCnxnFactory standaloneServerFactory;
163 while (true) {
164 try {
165 standaloneServerFactory = new NIOServerCnxnFactory();
166 standaloneServerFactory.configure(
167 new InetSocketAddress(tentativePort),
168 configuration.getInt(HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS,
169 1000));
170 } catch (BindException e) {
171 LOG.debug("Failed binding ZK Server to client port: " +
172 tentativePort);
173
174 tentativePort = selectClientPort();
175 continue;
176 }
177 break;
178 }
179
180
181 standaloneServerFactory.startup(server);
182 if (!waitForServerUp(tentativePort, CONNECTION_TIMEOUT)) {
183 throw new IOException("Waiting for startup of standalone server");
184 }
185
186
187 clientPortList.add(tentativePort);
188 standaloneServerFactoryList.add(standaloneServerFactory);
189 zooKeeperServers.add(server);
190 tentativePort++;
191 }
192
193
194 activeZKServerIndex = 0;
195 started = true;
196 clientPort = clientPortList.get(activeZKServerIndex);
197 LOG.info("Started MiniZK Cluster and connect 1 ZK server " +
198 "on client port: " + clientPort);
199 return clientPort;
200 }
201
202 private void recreateDir(File dir) throws IOException {
203 if (dir.exists()) {
204 if(!FileUtil.fullyDelete(dir)) {
205 throw new IOException("Could not delete zk base directory: " + dir);
206 }
207 }
208 try {
209 dir.mkdirs();
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
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
312 standaloneServerFactoryList.remove(backupZKServerIndex);
313 clientPortList.remove(backupZKServerIndex);
314 zooKeeperServers.remove(backupZKServerIndex);
315 LOG.info("Kill one backup ZK servers in the cluster " +
316 "on client port: " + clientPort);
317 }
318
319
320 private static boolean waitForServerDown(int port, long timeout) {
321 long start = System.currentTimeMillis();
322 while (true) {
323 try {
324 Socket sock = new Socket("localhost", port);
325 try {
326 OutputStream outstream = sock.getOutputStream();
327 outstream.write("stat".getBytes());
328 outstream.flush();
329 } finally {
330 sock.close();
331 }
332 } catch (IOException e) {
333 return true;
334 }
335
336 if (System.currentTimeMillis() > start + timeout) {
337 break;
338 }
339 try {
340 Thread.sleep(250);
341 } catch (InterruptedException e) {
342
343 }
344 }
345 return false;
346 }
347
348
349 private static boolean waitForServerUp(int port, long timeout) {
350 long start = System.currentTimeMillis();
351 while (true) {
352 try {
353 Socket sock = new Socket("localhost", port);
354 BufferedReader reader = null;
355 try {
356 OutputStream outstream = sock.getOutputStream();
357 outstream.write("stat".getBytes());
358 outstream.flush();
359
360 Reader isr = new InputStreamReader(sock.getInputStream());
361 reader = new BufferedReader(isr);
362 String line = reader.readLine();
363 if (line != null && line.startsWith("Zookeeper version:")) {
364 return true;
365 }
366 } finally {
367 sock.close();
368 if (reader != null) {
369 reader.close();
370 }
371 }
372 } catch (IOException e) {
373
374 LOG.info("server localhost:" + port + " not up " + e);
375 }
376
377 if (System.currentTimeMillis() > start + timeout) {
378 break;
379 }
380 try {
381 Thread.sleep(250);
382 } catch (InterruptedException e) {
383
384 }
385 }
386 return false;
387 }
388
389 public int getClientPort() {
390 return clientPort;
391 }
392 }