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