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