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