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;
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
32 import org.apache.commons.logging.Log;
33 import org.apache.commons.logging.LogFactory;
34 import org.apache.hadoop.fs.FileUtil;
35 import org.apache.zookeeper.server.NIOServerCnxn;
36 import org.apache.zookeeper.server.ZooKeeperServer;
37 import org.apache.zookeeper.server.persistence.FileTxnLog;
38
39
40
41
42
43
44 public class MiniZooKeeperCluster {
45 private static final Log LOG = LogFactory.getLog(MiniZooKeeperCluster.class);
46
47 private static final int TICK_TIME = 2000;
48 private static final int CONNECTION_TIMEOUT = 30000;
49
50 private boolean started;
51 private int clientPort = 21810;
52
53 private NIOServerCnxn.Factory standaloneServerFactory;
54 private int tickTime = 0;
55
56
57 public MiniZooKeeperCluster() {
58 this.started = false;
59 }
60
61 public void setClientPort(int clientPort) {
62 this.clientPort = clientPort;
63 }
64
65 public void setTickTime(int tickTime) {
66 this.tickTime = tickTime;
67 }
68
69
70 private static void setupTestEnv() {
71
72
73
74
75 System.setProperty("zookeeper.preAllocSize", "100");
76 FileTxnLog.setPreallocSize(100);
77 }
78
79
80
81
82
83
84
85 public int startup(File baseDir) throws IOException,
86 InterruptedException {
87
88 setupTestEnv();
89
90 shutdown();
91
92 File dir = new File(baseDir, "zookeeper").getAbsoluteFile();
93 recreateDir(dir);
94
95 int tickTimeToUse;
96 if (this.tickTime > 0) {
97 tickTimeToUse = this.tickTime;
98 } else {
99 tickTimeToUse = TICK_TIME;
100 }
101 ZooKeeperServer server = new ZooKeeperServer(dir, dir, tickTimeToUse);
102 while (true) {
103 try {
104 standaloneServerFactory =
105 new NIOServerCnxn.Factory(new InetSocketAddress(clientPort));
106 } catch (BindException e) {
107 LOG.info("Faild binding ZK Server to client port: " + clientPort);
108
109 clientPort++;
110 continue;
111 }
112 break;
113 }
114 standaloneServerFactory.startup(server);
115
116 if (!waitForServerUp(clientPort, CONNECTION_TIMEOUT)) {
117 throw new IOException("Waiting for startup of standalone server");
118 }
119
120 started = true;
121
122 return clientPort;
123 }
124
125 private void recreateDir(File dir) throws IOException {
126 if (dir.exists()) {
127 FileUtil.fullyDelete(dir);
128 }
129 try {
130 dir.mkdirs();
131 } catch (SecurityException e) {
132 throw new IOException("creating dir: " + dir, e);
133 }
134 }
135
136
137
138
139 public void shutdown() throws IOException {
140 if (!started) {
141 return;
142 }
143
144 standaloneServerFactory.shutdown();
145 if (!waitForServerDown(clientPort, CONNECTION_TIMEOUT)) {
146 throw new IOException("Waiting for shutdown of standalone server");
147 }
148
149 started = false;
150 }
151
152
153 private static boolean waitForServerDown(int port, long timeout) {
154 long start = System.currentTimeMillis();
155 while (true) {
156 try {
157 Socket sock = new Socket("localhost", port);
158 try {
159 OutputStream outstream = sock.getOutputStream();
160 outstream.write("stat".getBytes());
161 outstream.flush();
162 } finally {
163 sock.close();
164 }
165 } catch (IOException e) {
166 return true;
167 }
168
169 if (System.currentTimeMillis() > start + timeout) {
170 break;
171 }
172 try {
173 Thread.sleep(250);
174 } catch (InterruptedException e) {
175
176 }
177 }
178 return false;
179 }
180
181
182 private static boolean waitForServerUp(int port, long timeout) {
183 long start = System.currentTimeMillis();
184 while (true) {
185 try {
186 Socket sock = new Socket("localhost", port);
187 BufferedReader reader = null;
188 try {
189 OutputStream outstream = sock.getOutputStream();
190 outstream.write("stat".getBytes());
191 outstream.flush();
192
193 Reader isr = new InputStreamReader(sock.getInputStream());
194 reader = new BufferedReader(isr);
195 String line = reader.readLine();
196 if (line != null && line.startsWith("Zookeeper version:")) {
197 return true;
198 }
199 } finally {
200 sock.close();
201 if (reader != null) {
202 reader.close();
203 }
204 }
205 } catch (IOException e) {
206
207 LOG.info("server localhost:" + port + " not up " + e);
208 }
209
210 if (System.currentTimeMillis() > start + timeout) {
211 break;
212 }
213 try {
214 Thread.sleep(250);
215 } catch (InterruptedException e) {
216
217 }
218 }
219 return false;
220 }
221 }