http://git-wip-us.apache.org/repos/asf/zookeeper/blob/43d71c2e/zookeeper-server/src/test/java/org/apache/zookeeper/test/ChrootTest.java ---------------------------------------------------------------------- diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ChrootTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ChrootTest.java new file mode 100644 index 0000000..d65ed8d --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ChrootTest.java @@ -0,0 +1,136 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.ZooDefs.Ids; +import org.junit.Assert; +import org.junit.Test; + +public class ChrootTest extends ClientBase { + private static class MyWatcher implements Watcher { + private final String path; + private String eventPath; + private CountDownLatch latch = new CountDownLatch(1); + + public MyWatcher(String path) { + this.path = path; + } + public void process(WatchedEvent event) { + System.out.println("latch:" + path + " " + event.getPath()); + this.eventPath = event.getPath(); + latch.countDown(); + } + public boolean matches() throws InterruptedException { + if (!latch.await(CONNECTION_TIMEOUT, TimeUnit.MILLISECONDS)) { + Assert.fail("No watch received within timeout period " + path); + } + return path.equals(eventPath); + } + } + + @Test + public void testChrootSynchronous() + throws IOException, InterruptedException, KeeperException + { + ZooKeeper zk1 = createClient(); + try { + zk1.create("/ch1", null, Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + } finally { + if(zk1 != null) + zk1.close(); + } + ZooKeeper zk2 = createClient(hostPort + "/ch1"); + try { + Assert.assertEquals("/ch2", + zk2.create("/ch2", null, Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT)); + } finally { + if(zk2 != null) + zk2.close(); + } + + zk1 = createClient(); + zk2 = createClient(hostPort + "/ch1"); + try { + // check get + MyWatcher w1 = new MyWatcher("/ch1"); + Assert.assertNotNull(zk1.exists("/ch1", w1)); + MyWatcher w2 = new MyWatcher("/ch1/ch2"); + Assert.assertNotNull(zk1.exists("/ch1/ch2", w2)); + + MyWatcher w3 = new MyWatcher("/ch2"); + Assert.assertNotNull(zk2.exists("/ch2", w3)); + + // set watches on child + MyWatcher w4 = new MyWatcher("/ch1"); + zk1.getChildren("/ch1",w4); + MyWatcher w5 = new MyWatcher("/"); + zk2.getChildren("/",w5); + + // check set + zk1.setData("/ch1", "1".getBytes(), -1); + zk2.setData("/ch2", "2".getBytes(), -1); + + // check watches + Assert.assertTrue(w1.matches()); + Assert.assertTrue(w2.matches()); + Assert.assertTrue(w3.matches()); + + // check exceptions + try { + zk2.setData("/ch3", "3".getBytes(), -1); + } catch (KeeperException.NoNodeException e) { + Assert.assertEquals("/ch3", e.getPath()); + } + + Assert.assertTrue(Arrays.equals("1".getBytes(), + zk1.getData("/ch1", false, null))); + Assert.assertTrue(Arrays.equals("2".getBytes(), + zk1.getData("/ch1/ch2", false, null))); + Assert.assertTrue(Arrays.equals("2".getBytes(), + zk2.getData("/ch2", false, null))); + + // check delete + zk2.delete("/ch2", -1); + Assert.assertTrue(w4.matches()); + Assert.assertTrue(w5.matches()); + + zk1.delete("/ch1", -1); + Assert.assertNull(zk1.exists("/ch1", false)); + Assert.assertNull(zk1.exists("/ch1/ch2", false)); + Assert.assertNull(zk2.exists("/ch2", false)); + } finally { + if(zk1 != null) + zk1.close(); + if(zk2 != null) + zk2.close(); + } + } +}
http://git-wip-us.apache.org/repos/asf/zookeeper/blob/43d71c2e/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientBase.java ---------------------------------------------------------------------- diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientBase.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientBase.java new file mode 100644 index 0000000..8111099 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientBase.java @@ -0,0 +1,775 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.test; + +import static org.apache.zookeeper.client.FourLetterWordMain.send4LetterWord; + +import java.io.BufferedInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import javax.management.MBeanServerConnection; +import javax.management.ObjectName; + + +import org.apache.zookeeper.common.Time; +import org.apache.zookeeper.common.X509Exception.SSLContextException; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.TestableZooKeeper; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.Watcher.Event.KeeperState; +import org.apache.zookeeper.ZKTestCase; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.common.IOUtils; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.apache.zookeeper.server.ServerCnxnFactoryAccessor; +import org.apache.zookeeper.server.ZKDatabase; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.apache.zookeeper.server.persistence.FilePadding; +import org.apache.zookeeper.server.persistence.FileTxnLog; +import org.apache.zookeeper.server.quorum.QuorumPeer; +import org.apache.zookeeper.server.util.OSMXBean; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class ClientBase extends ZKTestCase { + protected static final Logger LOG = LoggerFactory.getLogger(ClientBase.class); + + public static int CONNECTION_TIMEOUT = 30000; + static final File BASETEST = + new File(System.getProperty("build.test.dir", "build")); + + protected String hostPort = "127.0.0.1:" + PortAssignment.unique(); + protected int maxCnxns = 0; + protected ServerCnxnFactory serverFactory = null; + protected File tmpDir = null; + + long initialFdCount; + + public ClientBase() { + super(); + } + + /** + * In general don't use this. Only use in the special case that you + * want to ignore results (for whatever reason) in your test. Don't + * use empty watchers in real code! + * + */ + protected static class NullWatcher implements Watcher { + public void process(WatchedEvent event) { /* nada */ } + } + + public static class CountdownWatcher implements Watcher { + // XXX this doesn't need to be volatile! (Should probably be final) + volatile CountDownLatch clientConnected; + // Set to true when connected to a read-only server, or a read-write (quorum) server. + volatile boolean connected; + // Set to true when connected to a quorum server. + volatile boolean syncConnected; + // Set to true when connected to a quorum server in read-only mode + volatile boolean readOnlyConnected; + + public CountdownWatcher() { + reset(); + } + synchronized public void reset() { + clientConnected = new CountDownLatch(1); + connected = false; + syncConnected = false; + readOnlyConnected = false; + } + synchronized public void process(WatchedEvent event) { + KeeperState state = event.getState(); + if (state == KeeperState.SyncConnected) { + connected = true; + syncConnected = true; + readOnlyConnected = false; + } else if (state == KeeperState.ConnectedReadOnly) { + connected = true; + syncConnected = false; + readOnlyConnected = true; + } else { + connected = false; + syncConnected = false; + readOnlyConnected = false; + } + + notifyAll(); + if (connected) { + clientConnected.countDown(); + } + } + synchronized public boolean isConnected() { + return connected; + } + synchronized public void waitForConnected(long timeout) + throws InterruptedException, TimeoutException + { + long expire = Time.currentElapsedTime() + timeout; + long left = timeout; + while(!connected && left > 0) { + wait(left); + left = expire - Time.currentElapsedTime(); + } + if (!connected) { + throw new TimeoutException("Failed to connect to ZooKeeper server."); + + } + } + synchronized public void waitForSyncConnected(long timeout) + throws InterruptedException, TimeoutException + { + long expire = Time.currentElapsedTime() + timeout; + long left = timeout; + while(!syncConnected && left > 0) { + wait(left); + left = expire - Time.currentElapsedTime(); + } + if (!syncConnected) { + throw new TimeoutException("Failed to connect to read-write ZooKeeper server."); + } + } + synchronized public void waitForReadOnlyConnected(long timeout) + throws InterruptedException, TimeoutException + { + long expire = System.currentTimeMillis() + timeout; + long left = timeout; + while(!readOnlyConnected && left > 0) { + wait(left); + left = expire - System.currentTimeMillis(); + } + if (!readOnlyConnected) { + throw new TimeoutException("Failed to connect in read-only mode to ZooKeeper server."); + } + } + synchronized public void waitForDisconnected(long timeout) + throws InterruptedException, TimeoutException + { + long expire = Time.currentElapsedTime() + timeout; + long left = timeout; + while(connected && left > 0) { + wait(left); + left = expire - Time.currentElapsedTime(); + } + if (connected) { + throw new TimeoutException("Did not disconnect"); + + } + } + } + + protected TestableZooKeeper createClient() + throws IOException, InterruptedException + { + return createClient(hostPort); + } + + protected TestableZooKeeper createClient(String hp) + throws IOException, InterruptedException + { + CountdownWatcher watcher = new CountdownWatcher(); + return createClient(watcher, hp); + } + + protected TestableZooKeeper createClient(CountdownWatcher watcher) + throws IOException, InterruptedException + { + return createClient(watcher, hostPort); + } + + private LinkedList<ZooKeeper> allClients; + private boolean allClientsSetup = false; + + protected TestableZooKeeper createClient(CountdownWatcher watcher, String hp) + throws IOException, InterruptedException + { + return createClient(watcher, hp, CONNECTION_TIMEOUT); + } + + protected TestableZooKeeper createClient(CountdownWatcher watcher, + String hp, int timeout) + throws IOException, InterruptedException + { + watcher.reset(); + TestableZooKeeper zk = new TestableZooKeeper(hp, timeout, watcher); + if (!watcher.clientConnected.await(timeout, TimeUnit.MILLISECONDS)) + { + Assert.fail("Unable to connect to server"); + } + synchronized(this) { + if (!allClientsSetup) { + LOG.error("allClients never setup"); + Assert.fail("allClients never setup"); + } + if (allClients != null) { + allClients.add(zk); + JMXEnv.ensureAll(getHexSessionId(zk.getSessionId())); + } else { + // test done - close the zk, not needed + zk.close(); + } + } + + return zk; + } + + public static class HostPort { + String host; + int port; + public HostPort(String host, int port) { + this.host = host; + this.port = port; + } + } + public static List<HostPort> parseHostPortList(String hplist) { + ArrayList<HostPort> alist = new ArrayList<HostPort>(); + for (String hp: hplist.split(",")) { + int idx = hp.lastIndexOf(':'); + String host = hp.substring(0, idx); + int port; + try { + port = Integer.parseInt(hp.substring(idx + 1)); + } catch(RuntimeException e) { + throw new RuntimeException("Problem parsing " + hp + e.toString()); + } + alist.add(new HostPort(host,port)); + } + return alist; + } + + public static boolean waitForServerUp(String hp, long timeout) { + return waitForServerUp(hp, timeout, false); + } + + public static boolean waitForServerUp(String hp, long timeout, boolean secure) { + long start = Time.currentElapsedTime(); + while (true) { + try { + // if there are multiple hostports, just take the first one + HostPort hpobj = parseHostPortList(hp).get(0); + String result = send4LetterWord(hpobj.host, hpobj.port, "stat", secure); + if (result.startsWith("Zookeeper version:") && + !result.contains("READ-ONLY")) { + return true; + } + } catch (IOException e) { + // ignore as this is expected + LOG.info("server {} not up", hp, e); + } catch (SSLContextException e) { + LOG.error("server {} not up", hp, e); + } + + if (Time.currentElapsedTime() > start + timeout) { + break; + } + try { + Thread.sleep(250); + } catch (InterruptedException e) { + // ignore + } + } + return false; + } + + public static boolean waitForServerDown(String hp, long timeout) { + return waitForServerDown(hp, timeout, false); + } + + public static boolean waitForServerDown(String hp, long timeout, boolean secure) { + long start = Time.currentElapsedTime(); + while (true) { + try { + HostPort hpobj = parseHostPortList(hp).get(0); + send4LetterWord(hpobj.host, hpobj.port, "stat", secure); + } catch (IOException e) { + return true; + } catch (SSLContextException e) { + return true; + } + + if (Time.currentElapsedTime() > start + timeout) { + break; + } + try { + Thread.sleep(250); + } catch (InterruptedException e) { + // ignore + } + } + return false; + } + + /** + * Return true if any of the states is achieved + */ + public static boolean waitForServerState(QuorumPeer qp, int timeout, + String... serverStates) { + long start = Time.currentElapsedTime(); + while (true) { + try { + Thread.sleep(250); + } catch (InterruptedException e) { + // ignore + } + for (String state : serverStates) { + if (qp.getServerState().equals(state)) { + return true; + } + } + if (Time.currentElapsedTime() > start + timeout) { + return false; + } + } + } + + static void verifyThreadTerminated(Thread thread, long millis) + throws InterruptedException + { + thread.join(millis); + if (thread.isAlive()) { + LOG.error("Thread " + thread.getName() + " : " + + Arrays.toString(thread.getStackTrace())); + Assert.assertFalse("thread " + thread.getName() + + " still alive after join", true); + } + } + + public static File createEmptyTestDir() throws IOException { + return createTmpDir(BASETEST, false); + } + + public static File createTmpDir() throws IOException { + return createTmpDir(BASETEST, true); + } + + static File createTmpDir(File parentDir, boolean createInitFile) throws IOException { + File tmpFile = File.createTempFile("test", ".junit", parentDir); + // don't delete tmpFile - this ensures we don't attempt to create + // a tmpDir with a duplicate name + File tmpDir = new File(tmpFile + ".dir"); + Assert.assertFalse(tmpDir.exists()); // never true if tmpfile does it's job + Assert.assertTrue(tmpDir.mkdirs()); + + // todo not every tmp directory needs this file + if (createInitFile) { + createInitializeFile(tmpDir); + } + + return tmpDir; + } + + public static void createInitializeFile(File dir) throws IOException { + File initFile = new File(dir, "initialize"); + if (!initFile.exists()) { + Assert.assertTrue(initFile.createNewFile()); + } + } + + private static int getPort(String hostPort) { + String[] split = hostPort.split(":"); + String portstr = split[split.length-1]; + String[] pc = portstr.split("/"); + if (pc.length > 1) { + portstr = pc[0]; + } + return Integer.parseInt(portstr); + } + + /** + * Starting the given server instance + */ + public static void startServerInstance(File dataDir, + ServerCnxnFactory factory, String hostPort, int serverId) throws IOException, + InterruptedException { + final int port = getPort(hostPort); + LOG.info("STARTING server instance 127.0.0.1:{}", port); + ZooKeeperServer zks = new ZooKeeperServer(dataDir, dataDir, 3000); + zks.setCreateSessionTrackerServerId(serverId); + factory.startup(zks); + Assert.assertTrue("waiting for server up", ClientBase.waitForServerUp( + "127.0.0.1:" + port, CONNECTION_TIMEOUT, factory.isSecure())); + } + + /** + * This method instantiates a new server. Starting of the server + * instance has been moved to a separate method + * {@link ClientBase#startServerInstance(File, ServerCnxnFactory, String, int)}. + * Because any exception on starting the server would leave the server + * running and the caller would not be able to shutdown the instance. This + * may affect other test cases. + * + * @return newly created server instance + * + * @see <a + * href="https://issues.apache.org/jira/browse/ZOOKEEPER-1852">ZOOKEEPER-1852</a> + * for more information. + */ + public static ServerCnxnFactory createNewServerInstance( + ServerCnxnFactory factory, String hostPort, int maxCnxns) + throws IOException, InterruptedException { + final int port = getPort(hostPort); + LOG.info("CREATING server instance 127.0.0.1:{}", port); + if (factory == null) { + factory = ServerCnxnFactory.createFactory(port, maxCnxns); + } + return factory; + } + + static void shutdownServerInstance(ServerCnxnFactory factory, + String hostPort) + { + if (factory != null) { + ZKDatabase zkDb = null; + { + ZooKeeperServer zs = getServer(factory); + if (zs != null) { + zkDb = zs.getZKDatabase(); + } + } + factory.shutdown(); + try { + if (zkDb != null) { + zkDb.close(); + } + } catch (IOException ie) { + LOG.warn("Error closing logs ", ie); + } + final int PORT = getPort(hostPort); + + Assert.assertTrue("waiting for server down", + ClientBase.waitForServerDown("127.0.0.1:" + PORT, + CONNECTION_TIMEOUT, + factory.isSecure())); + } + } + + /** + * Test specific setup + */ + public static void setupTestEnv() { + // during the tests we run with 100K prealloc in the logs. + // on windows systems prealloc of 64M was seen to take ~15seconds + // resulting in test Assert.failure (client timeout on first session). + // set env and directly in order to handle static init/gc issues + System.setProperty("zookeeper.preAllocSize", "100"); + FilePadding.setPreallocSize(100 * 1024); + } + + protected void setUpAll() throws Exception { + allClients = new LinkedList<ZooKeeper>(); + allClientsSetup = true; + } + + @Before + public void setUp() throws Exception { + setUpWithServerId(1); + } + + protected void setUpWithServerId(int serverId) throws Exception { + /* some useful information - log the number of fds used before + * and after a test is run. Helps to verify we are freeing resources + * correctly. Unfortunately this only works on unix systems (the + * only place sun has implemented as part of the mgmt bean api. + */ + OSMXBean osMbean = new OSMXBean(); + if (osMbean.getUnix() == true) { + initialFdCount = osMbean.getOpenFileDescriptorCount(); + LOG.info("Initial fdcount is: " + + initialFdCount); + } + + setupTestEnv(); + + JMXEnv.setUp(); + + setUpAll(); + + tmpDir = createTmpDir(BASETEST, true); + + startServer(serverId); + + LOG.info("Client test setup finished"); + } + + protected void startServer() throws Exception { + startServer(1); + } + + private void startServer(int serverId) throws Exception { + LOG.info("STARTING server"); + serverFactory = createNewServerInstance(serverFactory, hostPort, + maxCnxns); + startServerInstance(tmpDir, serverFactory, hostPort, serverId); + // ensure that server and data bean are registered + Set<ObjectName> children = JMXEnv.ensureParent("InMemoryDataTree", + "StandaloneServer_port"); + // Remove beans which are related to zk client sessions. Strong + // assertions cannot be done for these client sessions because + // registeration of these beans with server will happen only on their + // respective reconnection interval + verifyUnexpectedBeans(children); + } + + private void verifyUnexpectedBeans(Set<ObjectName> children) { + if (allClients != null) { + for (ZooKeeper zkc : allClients) { + Iterator<ObjectName> childItr = children.iterator(); + while (childItr.hasNext()) { + ObjectName clientBean = childItr.next(); + if (clientBean.toString().contains( + getHexSessionId(zkc.getSessionId()))) { + LOG.info("found name:" + zkc.getSessionId() + + " client bean:" + clientBean.toString()); + childItr.remove(); + } + } + } + } + for (ObjectName bean : children) { + LOG.info("unexpected:" + bean.toString()); + } + Assert.assertEquals("Unexpected bean exists!", 0, children.size()); + } + + /** + * Returns a string representation of the given long value session id + * + * @param sessionId + * long value of session id + * @return string representation of session id + */ + protected static String getHexSessionId(long sessionId) { + return "0x" + Long.toHexString(sessionId); + } + + protected void stopServer() throws Exception { + LOG.info("STOPPING server"); + shutdownServerInstance(serverFactory, hostPort); + serverFactory = null; + // ensure no beans are leftover + JMXEnv.ensureOnly(); + } + + + protected static ZooKeeperServer getServer(ServerCnxnFactory fac) { + ZooKeeperServer zs = ServerCnxnFactoryAccessor.getZkServer(fac); + + return zs; + } + + protected void tearDownAll() throws Exception { + synchronized (this) { + if (allClients != null) for (ZooKeeper zk : allClients) { + try { + if (zk != null) + zk.close(); + } catch (InterruptedException e) { + LOG.warn("ignoring interrupt", e); + } + } + allClients = null; + } + } + + @After + public void tearDown() throws Exception { + LOG.info("tearDown starting"); + + tearDownAll(); + + stopServer(); + + if (tmpDir != null) { + Assert.assertTrue("delete " + tmpDir.toString(), recursiveDelete(tmpDir)); + } + + // This has to be set to null when the same instance of this class is reused between test cases + serverFactory = null; + + JMXEnv.tearDown(); + + /* some useful information - log the number of fds used before + * and after a test is run. Helps to verify we are freeing resources + * correctly. Unfortunately this only works on unix systems (the + * only place sun has implemented as part of the mgmt bean api. + */ + OSMXBean osMbean = new OSMXBean(); + if (osMbean.getUnix() == true) { + long fdCount = osMbean.getOpenFileDescriptorCount(); + String message = "fdcount after test is: " + + fdCount + " at start it was " + initialFdCount; + LOG.info(message); + if (fdCount > initialFdCount) { + LOG.info("sleeping for 20 secs"); + //Thread.sleep(60000); + //assertTrue(message, fdCount <= initialFdCount); + } + } + } + + public static MBeanServerConnection jmxConn() throws IOException { + return JMXEnv.conn(); + } + + public static boolean recursiveDelete(File d) { + return TestUtils.deleteFileRecursively(d, true); + } + + public static void logAllStackTraces() { + StringBuilder sb = new StringBuilder(); + sb.append("Starting logAllStackTraces()\n"); + Map<Thread, StackTraceElement[]> threads = Thread.getAllStackTraces(); + for (Entry<Thread, StackTraceElement[]> e: threads.entrySet()) { + sb.append("Thread " + e.getKey().getName() + "\n"); + for (StackTraceElement elem: e.getValue()) { + sb.append("\tat " + elem + "\n"); + } + } + sb.append("Ending logAllStackTraces()\n"); + LOG.error(sb.toString()); + } + + /* + * Verify that all of the servers see the same number of nodes + * at the root + */ + void verifyRootOfAllServersMatch(String hostPort) + throws InterruptedException, KeeperException, IOException + { + String parts[] = hostPort.split(","); + + // run through till the counts no longer change on each server + // max 15 tries, with 2 second sleeps, so approx 30 seconds + int[] counts = new int[parts.length]; + int failed = 0; + for (int j = 0; j < 100; j++) { + int newcounts[] = new int[parts.length]; + int i = 0; + for (String hp : parts) { + try { + ZooKeeper zk = createClient(hp); + + try { + newcounts[i++] = zk.getChildren("/", false).size(); + } finally { + zk.close(); + } + } catch (Throwable t) { + failed++; + // if session creation Assert.fails dump the thread stack + // and try the next server + logAllStackTraces(); + } + } + + if (Arrays.equals(newcounts, counts)) { + LOG.info("Found match with array:" + + Arrays.toString(newcounts)); + counts = newcounts; + break; + } else { + counts = newcounts; + Thread.sleep(10000); + } + + // don't keep this up too long, will Assert.assert false below + if (failed > 10) { + break; + } + } + + // verify all the servers reporting same number of nodes + String logmsg = "node count not consistent{} {}"; + for (int i = 1; i < parts.length; i++) { + if (counts[i-1] != counts[i]) { + LOG.error(logmsg, Integer.valueOf(counts[i-1]), Integer.valueOf(counts[i])); + } else { + LOG.info(logmsg, Integer.valueOf(counts[i-1]), Integer.valueOf(counts[i])); + } + } + } + + public static String readFile(File file) throws IOException { + ByteArrayOutputStream os = new ByteArrayOutputStream(); + BufferedInputStream is = new BufferedInputStream(new FileInputStream(file)); + try { + IOUtils.copyBytes(is, os, 1024, true); + } finally { + is.close(); + } + return os.toString(); + } + + public static String join(String separator, Object[] parts) { + StringBuilder sb = new StringBuilder(); + boolean first = true; + for (Object part : parts) { + if (!first) { + sb.append(separator); + first = false; + } + sb.append(part); + } + return sb.toString(); + } + + public static ZooKeeper createZKClient(String cxnString) throws Exception { + return createZKClient(cxnString, CONNECTION_TIMEOUT); + } + + /** + * Returns ZooKeeper client after connecting to ZooKeeper Server. Session + * timeout is {@link #CONNECTION_TIMEOUT} + * + * @param cxnString + * connection string in the form of host:port + * @param sessionTimeout + * @throws IOException + * in cases of network failure + */ + public static ZooKeeper createZKClient(String cxnString, int sessionTimeout) throws IOException { + CountdownWatcher watcher = new CountdownWatcher(); + ZooKeeper zk = new ZooKeeper(cxnString, sessionTimeout, watcher); + try { + watcher.waitForConnected(CONNECTION_TIMEOUT); + } catch (InterruptedException | TimeoutException e) { + Assert.fail("ZooKeeper client can not connect to " + cxnString); + } + return zk; + } +} http://git-wip-us.apache.org/repos/asf/zookeeper/blob/43d71c2e/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientHammerTest.java ---------------------------------------------------------------------- diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientHammerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientHammerTest.java new file mode 100644 index 0000000..01cdf27 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientHammerTest.java @@ -0,0 +1,246 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.test; + +import java.io.IOException; +import java.util.Date; +import java.util.List; + +import org.apache.zookeeper.common.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.ZooDefs.Ids; +import org.junit.Assert; +import org.junit.Test; + +public class ClientHammerTest extends ClientBase { + protected static final Logger LOG = LoggerFactory.getLogger(ClientHammerTest.class); + + private static final long HAMMERTHREAD_LATENCY = 5; + + private static abstract class HammerThread extends Thread { + protected final int count; + protected volatile int current = 0; + + HammerThread(String name, int count) { + super(name); + this.count = count; + } + } + + private static class BasicHammerThread extends HammerThread { + private final ZooKeeper zk; + private final String prefix; + + BasicHammerThread(String name, ZooKeeper zk, String prefix, int count) { + super(name, count); + this.zk = zk; + this.prefix = prefix; + } + + public void run() { + byte b[] = new byte[256]; + try { + for (; current < count; current++) { + // Simulate a bit of network latency... + Thread.sleep(HAMMERTHREAD_LATENCY); + zk.create(prefix + current, b, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + } + } catch (Throwable t) { + LOG.error("Client create operation Assert.failed", t); + } finally { + try { + zk.close(); + } catch (InterruptedException e) { + LOG.warn("Unexpected", e); + } + } + } + } + + private static class SuperHammerThread extends HammerThread { + private final ClientHammerTest parent; + private final String prefix; + + SuperHammerThread(String name, ClientHammerTest parent, String prefix, + int count) + { + super(name, count); + this.parent = parent; + this.prefix = prefix; + } + + public void run() { + byte b[] = new byte[256]; + try { + for (; current < count; current++) { + ZooKeeper zk = parent.createClient(); + try { + zk.create(prefix + current, b, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + } finally { + try { + zk.close(); + } catch (InterruptedException e) { + LOG.warn("Unexpected", e); + } + } + } + } catch (Throwable t) { + LOG.error("Client create operation Assert.failed", t); + } + } + } + + /** + * Separate threads each creating a number of nodes. Each thread + * is using a non-shared (owned by thread) client for all node creations. + * @throws Throwable + */ + @Test + public void testHammerBasic() throws Throwable { + runHammer(10, 1000); + } + + public void runHammer(final int threadCount, final int childCount) + throws Throwable + { + try { + HammerThread[] threads = new HammerThread[threadCount]; + long start = Time.currentElapsedTime(); + for (int i = 0; i < threads.length; i++) { + ZooKeeper zk = createClient(); + String prefix = "/test-" + i; + zk.create(prefix, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + prefix += "/"; + HammerThread thread = + new BasicHammerThread("BasicHammerThread-" + i, zk, prefix, + childCount); + thread.start(); + + threads[i] = thread; + } + + verifyHammer(start, threads, childCount); + } catch (Throwable t) { + LOG.error("test Assert.failed", t); + throw t; + } + } + + /** + * Separate threads each creating a number of nodes. Each thread + * is creating a new client for each node creation. + * @throws Throwable + */ + @Test + public void testHammerSuper() throws Throwable { + try { + final int threadCount = 5; + final int childCount = 10; + + HammerThread[] threads = new HammerThread[threadCount]; + long start = Time.currentElapsedTime(); + for (int i = 0; i < threads.length; i++) { + String prefix = "/test-" + i; + { + ZooKeeper zk = createClient(); + try { + zk.create(prefix, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + } finally { + zk.close(); + } + } + prefix += "/"; + HammerThread thread = + new SuperHammerThread("SuperHammerThread-" + i, this, + prefix, childCount); + thread.start(); + + threads[i] = thread; + } + + verifyHammer(start, threads, childCount); + } catch (Throwable t) { + LOG.error("test Assert.failed", t); + throw t; + } + } + + public void verifyHammer(long start, HammerThread[] threads, int childCount) + throws IOException, InterruptedException, KeeperException + { + // look for the clients to finish their create operations + LOG.info("Starting check for completed hammers"); + int workingCount = threads.length; + for (int i = 0; i < 120; i++) { + Thread.sleep(10000); + for (HammerThread h : threads) { + if (!h.isAlive() || h.current == h.count) { + workingCount--; + } + } + if (workingCount == 0) { + break; + } + workingCount = threads.length; + } + if (workingCount > 0) { + for (HammerThread h : threads) { + LOG.warn(h.getName() + " never finished creation, current:" + + h.current); + } + } else { + LOG.info("Hammer threads completed creation operations"); + } + + for (HammerThread h : threads) { + final int safetyFactor = 3; + verifyThreadTerminated(h, + (long)threads.length * (long)childCount + * HAMMERTHREAD_LATENCY * (long)safetyFactor); + } + LOG.info(new Date() + " Total time " + + (Time.currentElapsedTime() - start)); + + ZooKeeper zk = createClient(); + try { + LOG.info("******************* Connected to ZooKeeper" + new Date()); + for (int i = 0; i < threads.length; i++) { + LOG.info("Doing thread: " + i + " " + new Date()); + List<String> children = + zk.getChildren("/test-" + i, false); + Assert.assertEquals(childCount, children.size()); + children = zk.getChildren("/test-" + i, false, null); + Assert.assertEquals(childCount, children.size()); + } + for (int i = 0; i < threads.length; i++) { + List<String> children = + zk.getChildren("/test-" + i, false); + Assert.assertEquals(childCount, children.size()); + children = zk.getChildren("/test-" + i, false, null); + Assert.assertEquals(childCount, children.size()); + } + } finally { + zk.close(); + } + } +} http://git-wip-us.apache.org/repos/asf/zookeeper/blob/43d71c2e/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientPortBindTest.java ---------------------------------------------------------------------- diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientPortBindTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientPortBindTest.java new file mode 100644 index 0000000..100d2a0 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientPortBindTest.java @@ -0,0 +1,106 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.test; + +import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT; + +import java.io.File; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.NetworkInterface; +import java.net.SocketException; +import java.util.Enumeration; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.ZKTestCase; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.junit.Assert; +import org.junit.Test; + +public class ClientPortBindTest extends ZKTestCase{ + protected static final Logger LOG = + LoggerFactory.getLogger(ClientPortBindTest.class); + + /** + * Verify that the server binds to the specified address + */ + @Test + public void testBindByAddress() throws Exception { + String bindAddress = null; + Enumeration<NetworkInterface> intfs = + NetworkInterface.getNetworkInterfaces(); + // if we have a loopback and it has an address use it + while(intfs.hasMoreElements()) { + NetworkInterface i = intfs.nextElement(); + try { + if (i.isLoopback()) { + Enumeration<InetAddress> addrs = i.getInetAddresses(); + while (addrs.hasMoreElements()) { + InetAddress a = addrs.nextElement(); + if(a.isLoopbackAddress()) { + bindAddress = a.getHostAddress(); + break; + } + } + } + } catch (SocketException se) { + LOG.warn("Couldn't find loopback interface: " + se.getMessage()); + } + } + if (bindAddress == null) { + LOG.warn("Unable to determine loop back address, skipping test"); + return; + } + final int PORT = PortAssignment.unique(); + + LOG.info("Using " + bindAddress + " as the bind address"); + final String HOSTPORT = bindAddress + ":" + PORT; + LOG.info("Using " + HOSTPORT + " as the host/port"); + + + File tmpDir = ClientBase.createTmpDir(); + + ClientBase.setupTestEnv(); + ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000); + + ServerCnxnFactory f = ServerCnxnFactory.createFactory( + new InetSocketAddress(bindAddress, PORT), -1); + f.startup(zks); + LOG.info("starting up the the server, waiting"); + + Assert.assertTrue("waiting for server up", + ClientBase.waitForServerUp(HOSTPORT, + CONNECTION_TIMEOUT)); + ZooKeeper zk = ClientBase.createZKClient(HOSTPORT); + try { + zk.close(); + } finally { + f.shutdown(); + zks.shutdown(); + + Assert.assertTrue("waiting for server down", + ClientBase.waitForServerDown(HOSTPORT, + CONNECTION_TIMEOUT)); + } + } +} http://git-wip-us.apache.org/repos/asf/zookeeper/blob/43d71c2e/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientRetryTest.java ---------------------------------------------------------------------- diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientRetryTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientRetryTest.java new file mode 100644 index 0000000..e53d911 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientRetryTest.java @@ -0,0 +1,73 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.zookeeper.test; + +import java.io.IOException; +import java.util.concurrent.TimeoutException; + +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.ZooKeeper.States; +import org.junit.Assert; +import org.junit.Test; + +public class ClientRetryTest extends ClientBase { + + @Override + public void setUp() throws Exception { + maxCnxns = 1; + super.setUp(); + } + /* + * This is a simple test - try to connect two clients to a server + * accepting a maximum of one connection from each address. Check that + * only one is accepted. Close that connection, and check that the other + * eventually connects. + * + * There is a possibility of a false positive here, as when zk2 is tested + * for having connected it might not have been given enough time, and finish + * connecting after the test is done. Since the + * server doesn't tell the client why it hasn't connected, there's no + * obvious way to detect the difference. + */ + @Test + public void testClientRetry() throws IOException, InterruptedException, TimeoutException{ + CountdownWatcher cdw1 = new CountdownWatcher(); + CountdownWatcher cdw2 = new CountdownWatcher(); + ZooKeeper zk = new ZooKeeper(hostPort, 10000, cdw1); + try { + cdw1.waitForConnected(CONNECTION_TIMEOUT); + ZooKeeper zk2 = new ZooKeeper(hostPort, 10000, cdw2); + try { + States s1 = zk.getState(); + States s2 = zk2.getState(); + Assert.assertSame(s1,States.CONNECTED); + Assert.assertSame(s2,States.CONNECTING); + cdw1.reset(); + zk.close(); + cdw1.waitForDisconnected(CONNECTION_TIMEOUT); + cdw2.waitForConnected(CONNECTION_TIMEOUT); + Assert.assertSame(zk2.getState(),States.CONNECTED); + } finally { + zk2.close(); + } + } finally { + zk.close(); + } + } +} + http://git-wip-us.apache.org/repos/asf/zookeeper/blob/43d71c2e/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java ---------------------------------------------------------------------- diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java new file mode 100644 index 0000000..08ffb4e --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSSLTest.java @@ -0,0 +1,132 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * <p/> + * http://www.apache.org/licenses/LICENSE-2.0 + * <p/> + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * + */ +package org.apache.zookeeper.test; + + +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.client.ZKClientConfig; +import org.apache.zookeeper.common.ClientX509Util; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.apache.zookeeper.server.quorum.QuorumPeerTestBase; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class ClientSSLTest extends QuorumPeerTestBase { + + private ClientX509Util clientX509Util = new ClientX509Util(); + + @Before + public void setup() { + String testDataPath = System.getProperty("test.data.dir", "build/test/data"); + System.setProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY, "org.apache.zookeeper.server.NettyServerCnxnFactory"); + System.setProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET, "org.apache.zookeeper.ClientCnxnSocketNetty"); + System.setProperty(ZKClientConfig.SECURE_CLIENT, "true"); + System.setProperty(clientX509Util.getSslKeystoreLocationProperty(), testDataPath + "/ssl/testKeyStore.jks"); + System.setProperty(clientX509Util.getSslKeystorePasswdProperty(), "testpass"); + System.setProperty(clientX509Util.getSslTruststoreLocationProperty(), testDataPath + "/ssl/testTrustStore.jks"); + System.setProperty(clientX509Util.getSslTruststorePasswdProperty(), "testpass"); + } + + @After + public void teardown() throws Exception { + System.clearProperty(ServerCnxnFactory.ZOOKEEPER_SERVER_CNXN_FACTORY); + System.clearProperty(ZKClientConfig.ZOOKEEPER_CLIENT_CNXN_SOCKET); + System.clearProperty(ZKClientConfig.SECURE_CLIENT); + System.clearProperty(clientX509Util.getSslKeystoreLocationProperty()); + System.clearProperty(clientX509Util.getSslKeystorePasswdProperty()); + System.clearProperty(clientX509Util.getSslTruststoreLocationProperty()); + System.clearProperty(clientX509Util.getSslTruststorePasswdProperty()); + } + + /** + * This test checks that client <-> server SSL works in cluster setup of ZK servers, which includes: + * 1. setting "secureClientPort" in "zoo.cfg" file. + * 2. setting jvm flags for serverCnxn, keystore, truststore. + * Finally, a zookeeper client should be able to connect to the secure port and + * communicate with server via secure connection. + * <p/> + * Note that in this test a ZK server has two ports -- clientPort and secureClientPort. + */ + @Test + public void testClientServerSSL() throws Exception { + final int SERVER_COUNT = 3; + final int clientPorts[] = new int[SERVER_COUNT]; + final Integer secureClientPorts[] = new Integer[SERVER_COUNT]; + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < SERVER_COUNT; i++) { + clientPorts[i] = PortAssignment.unique(); + secureClientPorts[i] = PortAssignment.unique(); + String server = String.format("server.%d=localhost:%d:%d:participant;localhost:%d", + i, PortAssignment.unique(), PortAssignment.unique(), clientPorts[i]); + sb.append(server + "\n"); + } + String quorumCfg = sb.toString(); + + + MainThread[] mt = new MainThread[SERVER_COUNT]; + for (int i = 0; i < SERVER_COUNT; i++) { + mt[i] = new MainThread(i, quorumCfg, secureClientPorts[i], true); + mt[i].start(); + } + + // Servers have been set up. Now go test if secure connection is successful. + for (int i = 0; i < SERVER_COUNT; i++) { + Assert.assertTrue("waiting for server " + i + " being up", + ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], TIMEOUT)); + + ZooKeeper zk = ClientBase.createZKClient("127.0.0.1:" + secureClientPorts[i], TIMEOUT); + // Do a simple operation to make sure the connection is fine. + zk.create("/test", "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.delete("/test", -1); + zk.close(); + } + + for (int i = 0; i < mt.length; i++) { + mt[i].shutdown(); + } + } + + + /** + * Developers might use standalone mode (which is the default for one server). + * This test checks SSL works in standalone mode of ZK server. + * <p/> + * Note that in this test the Zk server has only secureClientPort + */ + @Test + public void testSecureStandaloneServer() throws Exception { + Integer secureClientPort = PortAssignment.unique(); + MainThread mt = new MainThread(MainThread.UNSET_MYID, "", secureClientPort, false); + mt.start(); + + ZooKeeper zk = ClientBase.createZKClient("127.0.0.1:" + secureClientPort, TIMEOUT); + zk.create("/test", "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.delete("/test", -1); + zk.close(); + mt.shutdown(); + } +} http://git-wip-us.apache.org/repos/asf/zookeeper/blob/43d71c2e/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSkipACLTest.java ---------------------------------------------------------------------- diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSkipACLTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSkipACLTest.java new file mode 100644 index 0000000..2c57ff7 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ClientSkipACLTest.java @@ -0,0 +1,34 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.zookeeper.test; + +import org.junit.AfterClass; +import org.junit.BeforeClass; + +public class ClientSkipACLTest extends ClientTest { + @BeforeClass + static public void setup() { + System.setProperty("zookeeper.skipACL", "yes"); + } + + @AfterClass + static public void teardown() { + System.clearProperty("zookeeper.skipACL"); + } +} \ No newline at end of file