View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.zookeeper;
20  
21  import java.io.BufferedReader;
22  import java.io.File;
23  import java.io.InterruptedIOException;
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.hbase.classification.InterfaceAudience;
38  import org.apache.hadoop.hbase.classification.InterfaceStability;
39  import org.apache.hadoop.conf.Configuration;
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  import com.google.common.annotations.VisibleForTesting;
46  
47  /**
48   * TODO: Most of the code in this class is ripped from ZooKeeper tests. Instead
49   * of redoing it, we should contribute updates to their code which let us more
50   * easily access testing helper objects.
51   */
52  @InterfaceAudience.Public
53  @InterfaceStability.Evolving
54  public class MiniZooKeeperCluster {
55    private static final Log LOG = LogFactory.getLog(MiniZooKeeperCluster.class);
56  
57    private static final int TICK_TIME = 2000;
58    private static final int DEFAULT_CONNECTION_TIMEOUT = 30000;
59    private int connectionTimeout;
60  
61    private boolean started;
62  
63    /** The default port. If zero, we use a random port. */
64    private int defaultClientPort = 0;
65  
66    private List<NIOServerCnxnFactory> standaloneServerFactoryList;
67    private List<ZooKeeperServer> zooKeeperServers;
68    private List<Integer> clientPortList;
69  
70    private int activeZKServerIndex;
71    private int tickTime = 0;
72  
73    private Configuration configuration;
74  
75    public MiniZooKeeperCluster() {
76      this(new Configuration());
77    }
78  
79    public MiniZooKeeperCluster(Configuration configuration) {
80      this.started = false;
81      this.configuration = configuration;
82      activeZKServerIndex = -1;
83      zooKeeperServers = new ArrayList<ZooKeeperServer>();
84      clientPortList = new ArrayList<Integer>();
85      standaloneServerFactoryList = new ArrayList<NIOServerCnxnFactory>();
86      connectionTimeout = configuration.getInt(HConstants.ZK_SESSION_TIMEOUT + ".localHBaseCluster",
87        DEFAULT_CONNECTION_TIMEOUT);
88    }
89  
90    /**
91     * Add a client port to the list.
92     *
93     * @param clientPort the specified port
94     */
95    public void addClientPort(int clientPort) {
96      clientPortList.add(clientPort);
97    }
98  
99    /**
100    * Get the list of client ports.
101    * @return clientPortList the client port list
102    */
103   @VisibleForTesting
104   public List<Integer> getClientPortList() {
105     return clientPortList;
106   }
107 
108   /**
109    * Check whether the client port in a specific position of the client port list is valid.
110    *
111    * @param index the specified position
112    */
113   private boolean hasValidClientPortInList(int index) {
114     return (clientPortList.size() > index && clientPortList.get(index) > 0);
115   }
116 
117   public void setDefaultClientPort(int clientPort) {
118     if (clientPort <= 0) {
119       throw new IllegalArgumentException("Invalid default ZK client port: "
120           + clientPort);
121     }
122     this.defaultClientPort = clientPort;
123   }
124 
125   /**
126    * Selects a ZK client port.
127    *
128    * @param seedPort the seed port to start with; -1 means first time.
129    * @Returns a valid and unused client port
130    */
131   private int selectClientPort(int seedPort) {
132     int i;
133     int returnClientPort = seedPort + 1;
134     if (returnClientPort == 0) {
135       // If the new port is invalid, find one - starting with the default client port.
136       // If the default client port is not specified, starting with a random port.
137       // The random port is selected from the range between 49152 to 65535. These ports cannot be
138       // registered with IANA and are intended for dynamic allocation (see http://bit.ly/dynports).
139       if (defaultClientPort > 0) {
140         returnClientPort = defaultClientPort;
141       } else {
142         returnClientPort = 0xc000 + new Random().nextInt(0x3f00);
143       }
144     }
145     // Make sure that the port is unused.
146     while (true) {
147       for (i = 0; i < clientPortList.size(); i++) {
148         if (returnClientPort == clientPortList.get(i)) {
149           // Already used. Update the port and retry.
150           returnClientPort++;
151           break;
152         }
153       }
154       if (i == clientPortList.size()) {
155         break; // found a unused port, exit
156       }
157     }
158     return returnClientPort;
159   }
160 
161   public void setTickTime(int tickTime) {
162     this.tickTime = tickTime;
163   }
164 
165   public int getBackupZooKeeperServerNum() {
166     return zooKeeperServers.size()-1;
167   }
168 
169   public int getZooKeeperServerNum() {
170     return zooKeeperServers.size();
171   }
172 
173   // / XXX: From o.a.zk.t.ClientBase
174   private static void setupTestEnv() {
175     // during the tests we run with 100K prealloc in the logs.
176     // on windows systems prealloc of 64M was seen to take ~15seconds
177     // resulting in test failure (client timeout on first session).
178     // set env and directly in order to handle static init/gc issues
179     System.setProperty("zookeeper.preAllocSize", "100");
180     FileTxnLog.setPreallocSize(100 * 1024);
181   }
182 
183   public int startup(File baseDir) throws IOException, InterruptedException {
184     int numZooKeeperServers = clientPortList.size();
185     if (numZooKeeperServers == 0) {
186       numZooKeeperServers = 1; // need at least 1 ZK server for testing
187     }
188     return startup(baseDir, numZooKeeperServers);
189   }
190 
191   /**
192    * @param baseDir
193    * @param numZooKeeperServers
194    * @return ClientPort server bound to, -1 if there was a
195    *         binding problem and we couldn't pick another port.
196    * @throws IOException
197    * @throws InterruptedException
198    */
199   public int startup(File baseDir, int numZooKeeperServers) throws IOException,
200       InterruptedException {
201     if (numZooKeeperServers <= 0)
202       return -1;
203 
204     setupTestEnv();
205     shutdown();
206 
207     int tentativePort = -1; // the seed port
208     int currentClientPort;
209 
210     // running all the ZK servers
211     for (int i = 0; i < numZooKeeperServers; i++) {
212       File dir = new File(baseDir, "zookeeper_"+i).getAbsoluteFile();
213       createDir(dir);
214       int tickTimeToUse;
215       if (this.tickTime > 0) {
216         tickTimeToUse = this.tickTime;
217       } else {
218         tickTimeToUse = TICK_TIME;
219       }
220 
221       // Set up client port - if we have already had a list of valid ports, use it.
222       if (hasValidClientPortInList(i)) {
223         currentClientPort = clientPortList.get(i);
224       } else {
225         tentativePort = selectClientPort(tentativePort); // update the seed
226         currentClientPort = tentativePort;
227       }
228 
229       ZooKeeperServer server = new ZooKeeperServer(dir, dir, tickTimeToUse);
230       NIOServerCnxnFactory standaloneServerFactory;
231       while (true) {
232         try {
233           standaloneServerFactory = new NIOServerCnxnFactory();
234           standaloneServerFactory.configure(
235             new InetSocketAddress(currentClientPort),
236             configuration.getInt(HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS, 1000));
237         } catch (BindException e) {
238           LOG.debug("Failed binding ZK Server to client port: " +
239               currentClientPort, e);
240           // We're told to use some port but it's occupied, fail
241           if (hasValidClientPortInList(i)) {
242             return -1;
243           }
244           // This port is already in use, try to use another.
245           tentativePort = selectClientPort(tentativePort);
246           currentClientPort = tentativePort;
247           continue;
248         }
249         break;
250       }
251 
252       // Start up this ZK server
253       standaloneServerFactory.startup(server);
254       // Runs a 'stat' against the servers.
255       if (!waitForServerUp(currentClientPort, connectionTimeout)) {
256         throw new IOException("Waiting for startup of standalone server");
257       }
258 
259       // We have selected a port as a client port.  Update clientPortList if necessary.
260       if (clientPortList.size() <= i) { // it is not in the list, add the port
261         clientPortList.add(currentClientPort);
262       }
263       else if (clientPortList.get(i) <= 0) { // the list has invalid port, update with valid port
264         clientPortList.remove(i);
265         clientPortList.add(i, currentClientPort);
266       }
267 
268       standaloneServerFactoryList.add(standaloneServerFactory);
269       zooKeeperServers.add(server);
270     }
271 
272     // set the first one to be active ZK; Others are backups
273     activeZKServerIndex = 0;
274     started = true;
275     int clientPort = clientPortList.get(activeZKServerIndex);
276     LOG.info("Started MiniZooKeeperCluster and ran successful 'stat' " +
277         "on client port=" + clientPort);
278     return clientPort;
279   }
280 
281   private void createDir(File dir) throws IOException {
282     try {
283       if (!dir.exists()) {
284         dir.mkdirs();
285       }
286     } catch (SecurityException e) {
287       throw new IOException("creating dir: " + dir, e);
288     }
289   }
290 
291   /**
292    * @throws IOException
293    */
294   public void shutdown() throws IOException {
295     // shut down all the zk servers
296     for (int i = 0; i < standaloneServerFactoryList.size(); i++) {
297       NIOServerCnxnFactory standaloneServerFactory =
298         standaloneServerFactoryList.get(i);
299       int clientPort = clientPortList.get(i);
300 
301       standaloneServerFactory.shutdown();
302       if (!waitForServerDown(clientPort, connectionTimeout)) {
303         throw new IOException("Waiting for shutdown of standalone server");
304       }
305     }
306     standaloneServerFactoryList.clear();
307 
308     for (ZooKeeperServer zkServer: zooKeeperServers) {
309       //explicitly close ZKDatabase since ZookeeperServer does not close them
310       zkServer.getZKDatabase().close();
311     }
312     zooKeeperServers.clear();
313 
314     // clear everything
315     if (started) {
316       started = false;
317       activeZKServerIndex = 0;
318       clientPortList.clear();
319       LOG.info("Shutdown MiniZK cluster with all ZK servers");
320     }
321   }
322 
323   /**@return clientPort return clientPort if there is another ZK backup can run
324    *         when killing the current active; return -1, if there is no backups.
325    * @throws IOException
326    * @throws InterruptedException
327    */
328   public int killCurrentActiveZooKeeperServer() throws IOException,
329                                         InterruptedException {
330     if (!started || activeZKServerIndex < 0) {
331       return -1;
332     }
333 
334     // Shutdown the current active one
335     NIOServerCnxnFactory standaloneServerFactory =
336       standaloneServerFactoryList.get(activeZKServerIndex);
337     int clientPort = clientPortList.get(activeZKServerIndex);
338 
339     standaloneServerFactory.shutdown();
340     if (!waitForServerDown(clientPort, connectionTimeout)) {
341       throw new IOException("Waiting for shutdown of standalone server");
342     }
343 
344     zooKeeperServers.get(activeZKServerIndex).getZKDatabase().close();
345 
346     // remove the current active zk server
347     standaloneServerFactoryList.remove(activeZKServerIndex);
348     clientPortList.remove(activeZKServerIndex);
349     zooKeeperServers.remove(activeZKServerIndex);
350     LOG.info("Kill the current active ZK servers in the cluster " +
351         "on client port: " + clientPort);
352 
353     if (standaloneServerFactoryList.size() == 0) {
354       // there is no backup servers;
355       return -1;
356     }
357     clientPort = clientPortList.get(activeZKServerIndex);
358     LOG.info("Activate a backup zk server in the cluster " +
359         "on client port: " + clientPort);
360     // return the next back zk server's port
361     return clientPort;
362   }
363 
364   /**
365    * Kill one back up ZK servers
366    * @throws IOException
367    * @throws InterruptedException
368    */
369   public void killOneBackupZooKeeperServer() throws IOException,
370                                         InterruptedException {
371     if (!started || activeZKServerIndex < 0 ||
372         standaloneServerFactoryList.size() <= 1) {
373       return ;
374     }
375 
376     int backupZKServerIndex = activeZKServerIndex+1;
377     // Shutdown the current active one
378     NIOServerCnxnFactory standaloneServerFactory =
379       standaloneServerFactoryList.get(backupZKServerIndex);
380     int clientPort = clientPortList.get(backupZKServerIndex);
381 
382     standaloneServerFactory.shutdown();
383     if (!waitForServerDown(clientPort, connectionTimeout)) {
384       throw new IOException("Waiting for shutdown of standalone server");
385     }
386 
387     zooKeeperServers.get(backupZKServerIndex).getZKDatabase().close();
388 
389     // remove this backup zk server
390     standaloneServerFactoryList.remove(backupZKServerIndex);
391     clientPortList.remove(backupZKServerIndex);
392     zooKeeperServers.remove(backupZKServerIndex);
393     LOG.info("Kill one backup ZK servers in the cluster " +
394         "on client port: " + clientPort);
395   }
396 
397   // XXX: From o.a.zk.t.ClientBase
398   private static boolean waitForServerDown(int port, long timeout) throws IOException {
399     long start = System.currentTimeMillis();
400     while (true) {
401       try {
402         Socket sock = new Socket("localhost", port);
403         try {
404           OutputStream outstream = sock.getOutputStream();
405           outstream.write("stat".getBytes());
406           outstream.flush();
407         } finally {
408           sock.close();
409         }
410       } catch (IOException e) {
411         return true;
412       }
413 
414       if (System.currentTimeMillis() > start + timeout) {
415         break;
416       }
417       try {
418         Thread.sleep(250);
419       } catch (InterruptedException e) {
420         throw (InterruptedIOException)new InterruptedIOException().initCause(e);
421       }
422     }
423     return false;
424   }
425 
426   // XXX: From o.a.zk.t.ClientBase
427   private static boolean waitForServerUp(int port, long timeout) throws IOException {
428     long start = System.currentTimeMillis();
429     while (true) {
430       try {
431         Socket sock = new Socket("localhost", port);
432         BufferedReader reader = null;
433         try {
434           OutputStream outstream = sock.getOutputStream();
435           outstream.write("stat".getBytes());
436           outstream.flush();
437 
438           Reader isr = new InputStreamReader(sock.getInputStream());
439           reader = new BufferedReader(isr);
440           String line = reader.readLine();
441           if (line != null && line.startsWith("Zookeeper version:")) {
442             return true;
443           }
444         } finally {
445           sock.close();
446           if (reader != null) {
447             reader.close();
448           }
449         }
450       } catch (IOException e) {
451         // ignore as this is expected
452         LOG.info("server localhost:" + port + " not up " + e);
453       }
454 
455       if (System.currentTimeMillis() > start + timeout) {
456         break;
457       }
458       try {
459         Thread.sleep(250);
460       } catch (InterruptedException e) {
461         throw (InterruptedIOException)new InterruptedIOException().initCause(e);
462       }
463     }
464     return false;
465   }
466 
467   public int getClientPort() {
468     return activeZKServerIndex < 0 || activeZKServerIndex >= clientPortList.size() ? -1
469         : clientPortList.get(activeZKServerIndex);
470   }
471 }