Author: mahadev
Date: Thu Jul 16 22:09:40 2009
New Revision: 794887
URL: http://svn.apache.org/viewvc?rev=794887&view=rev
Log:
ZOOKEEPER-473. cleanup junit tests to eliminate false positives due to socket reuse and failure to close client (phunt via mahadev)
Added:
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/PortAssignment.java
Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/CRCTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DataTreeUnitTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DeserializationPerfTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/InvalidSnapshotTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/SerializationPerfTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/ZooKeeperServerTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ACLTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootClientTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientRetry.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/DataTreeTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/EventTypeTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLENewEpochTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLERestartTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLETest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/HierarchicalQuorumTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/IntegrityCheck.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/InvalidSnapshotTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LETest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/NullDataTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/OOMTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/PurgeTxnTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/QuorumBase.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/QuorumQuotaTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/QuorumTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/RecoveryTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/RepeatStartupTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SessionTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/SyncCallTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/TestHammer.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/UpgradeTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatchedEventTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatcherFuncTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/WatcherTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperQuotaTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperTestClient.java
Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Thu Jul 16 22:09:40 2009
@@ -9,6 +9,8 @@
BUGFIXES:
IMPROVEMENTS:
+ ZOOKEEPER-473. cleanup junit tests to eliminate false positives due to
+"socket reuse" and failure to close client (phunt via mahadev)
NEW FEATURES:
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/ClientCnxn.java Thu Jul 16 22:09:40 2009
@@ -76,18 +76,18 @@
*/
public class ClientCnxn {
private static final Logger LOG = Logger.getLogger(ClientCnxn.class);
-
+
/** This controls whether automatic watch resetting is enabled.
* Clients automatically reset watches during session reconnect, this
* option allows the client to turn off this behavior by setting
* the environment variable "zookeeper.disableAutoWatchReset" to "true" */
private static boolean disableAutoWatchReset;
-
+
public static final int packetLen;
static {
- // this var should not be public, but otw there is no easy way
+ // this var should not be public, but otw there is no easy way
// to test
- disableAutoWatchReset =
+ disableAutoWatchReset =
Boolean.getBoolean("zookeeper.disableAutoWatchReset");
LOG.info("zookeeper.disableAutoWatchReset is " + disableAutoWatchReset);
packetLen = Integer.getInteger("jute.maxbuffer", 4096 * 1024);
@@ -105,7 +105,7 @@
byte data[];
}
-
+
private ArrayList<AuthData> authInfo = new ArrayList<AuthData>();
/**
@@ -141,7 +141,7 @@
final EventThread eventThread;
final Selector selector = Selector.open();
-
+
/**
* Set to true when close is called. Latches the connection such that we
* don't attempt to re-connect to the server if in the middle of closing the
@@ -153,7 +153,7 @@
public long getSessionId() {
return sessionId;
}
-
+
public byte[] getSessionPasswd() {
return sessionPasswd;
}
@@ -329,7 +329,7 @@
sendThread = new SendThread();
eventThread = new EventThread();
}
-
+
/**
* tests use this to check on reset of watches
* @return if the auto reset of watches are disabled
@@ -360,15 +360,15 @@
private static class WatcherSetEventPair {
private final Set<Watcher> watchers;
private final WatchedEvent event;
-
+
public WatcherSetEventPair(Set<Watcher> watchers, WatchedEvent event) {
this.watchers = watchers;
this.event = event;
}
}
-
+
class EventThread extends Thread {
- private final LinkedBlockingQueue<Object> waitingEvents =
+ private final LinkedBlockingQueue<Object> waitingEvents =
new LinkedBlockingQueue<Object>();
/** This is really the queued session state until the event
@@ -382,9 +382,9 @@
setUncaughtExceptionHandler(uncaughtExceptionHandler);
setDaemon(true);
}
-
+
public void queueEvent(WatchedEvent event) {
- if (event.getType() == EventType.None
+ if (event.getType() == EventType.None
&& sessionState == event.getState()) {
return;
}
@@ -398,7 +398,7 @@
// queue the pair (watch set & event) for later processing
waitingEvents.add(pair);
}
-
+
public void queuePacket(Packet packet) {
waitingEvents.add(packet);
}
@@ -510,7 +510,7 @@
} catch (InterruptedException e) {
LOG.error("Event thread exiting due to interruption", e);
}
-
+
LOG.info("EventThread shut down");
}
}
@@ -609,7 +609,7 @@
if (LOG.isDebugEnabled()) {
LOG.debug("Got ping response for sessionid:0x"
+ Long.toHexString(sessionId)
- + " after "
+ + " after "
+ ((System.nanoTime() - lastPingSentNs) / 1000000)
+ "ms");
}
@@ -640,7 +640,7 @@
LOG.debug("Got " + we + " for sessionid 0x"
+ Long.toHexString(sessionId));
}
-
+
eventThread.queueEvent( we );
return;
}
@@ -775,7 +775,7 @@
}
SendThread() {
- super(currentThread().getName() + "-SendThread");
+ super(currentThread().getName() + "-SendThread()");
zooKeeper.state = States.CONNECTING;
setUncaughtExceptionHandler(uncaughtExceptionHandler);
setDaemon(true);
@@ -867,6 +867,8 @@
sock.socket().setSoLinger(false, -1);
sock.socket().setTcpNoDelay(true);
LOG.info("Attempting connection to server " + addr);
+ setName(getName().replaceAll("\\(.*\\)",
+ "(" + addr.getHostName() + ":" + addr.getPort() + ")"));
sockKey = sock.register(selector, SelectionKey.OP_CONNECT);
if (sock.connect(addr)) {
primeConnection(sockKey);
@@ -964,7 +966,7 @@
+ " : " + e.getMessage());
break;
} else {
- LOG.warn("Exception closing session 0x"
+ LOG.warn("Exception closing session 0x"
+ Long.toHexString(getSessionId()) + " to "
+ sockKey, e);
cleanup();
@@ -974,7 +976,7 @@
Event.KeeperState.Disconnected,
null));
}
-
+
now = System.currentTimeMillis();
lastHeard = now;
lastSend = now;
@@ -1051,7 +1053,7 @@
* behavior.
*/
public void disconnect() {
- LOG.info("Disconnecting ClientCnxn for session: 0x"
+ LOG.info("Disconnecting ClientCnxn for session: 0x"
+ Long.toHexString(getSessionId()));
sendThread.close();
@@ -1061,19 +1063,19 @@
/**
* Close the connection, which includes; send session disconnect to the
* server, shutdown the send/event threads.
- *
+ *
* @throws IOException
*/
public void close() throws IOException {
- LOG.info("Closing ClientCnxn for session: 0x"
+ LOG.info("Closing ClientCnxn for session: 0x"
+ Long.toHexString(getSessionId()));
closing = true;
-
+
try {
RequestHeader h = new RequestHeader();
h.setType(ZooDefs.OpCode.closeSession);
-
+
submitRequest(h, null, null, null);
} catch (InterruptedException e) {
// ignore, close the send/event threads
Modified: hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java (original)
+++ hadoop/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/NIOServerCnxn.java Thu Jul 16 22:09:40 2009
@@ -89,13 +89,13 @@
ByteBuffer directBuffer = ByteBuffer.allocateDirect(64 * 1024);
HashSet<NIOServerCnxn> cnxns = new HashSet<NIOServerCnxn>();
- HashMap<InetAddress, Set<NIOServerCnxn>> ipMap = new HashMap<InetAddress, Set<NIOServerCnxn>>( );
+ HashMap<InetAddress, Set<NIOServerCnxn>> ipMap = new HashMap<InetAddress, Set<NIOServerCnxn>>( );
int outstandingLimit = 1;
- int maxClientCnxns = 10;
-
-
+ int maxClientCnxns = 10;
+
+
/**
* Construct a new server connection factory which will accept an unlimited number
* of concurrent connections from each client (up to the file descriptor
@@ -106,13 +106,13 @@
public Factory(int port) throws IOException {
this(port,0);
}
-
-
+
+
/**
* Constructs a new server connection factory where the number of concurrent connections
* from a single IP address is limited to maxcc (or unlimited if 0).
* startup(zks) must be called subsequently.
- * @param port - the port to listen on for connections.
+ * @param port - the port to listen on for connections.
* @param maxcc - the number of concurrent connections allowed from a single client.
* @throws IOException
*/
@@ -122,9 +122,10 @@
maxClientCnxns = maxcc;
this.ss = ServerSocketChannel.open();
ss.socket().setReuseAddress(true);
+ LOG.info("binding to port " + port);
ss.socket().bind(new InetSocketAddress(port));
ss.configureBlocking(false);
- ss.register(selector, SelectionKey.OP_ACCEPT);
+ ss.register(selector, SelectionKey.OP_ACCEPT);
}
@Override
@@ -175,7 +176,7 @@
}
s.add(cnxn);
ipMap.put(addr,s);
- }
+ }
}
}
@@ -189,8 +190,8 @@
if (s == null) return 0;
return s.size();
}
-
- public void run() {
+
+ public void run() {
while (!ss.socket().isClosed()) {
try {
selector.select(1000);
@@ -204,13 +205,13 @@
for (SelectionKey k : selectedList) {
if ((k.readyOps() & SelectionKey.OP_ACCEPT) != 0) {
SocketChannel sc = ((ServerSocketChannel) k
- .channel()).accept();
+ .channel()).accept();
InetAddress ia = sc.socket().getInetAddress();
- int cnxncount = getClientCnxnCount(ia);
- if (maxClientCnxns > 0 && cnxncount >= maxClientCnxns){
- LOG.warn("Too many connections from " + ia
+ int cnxncount = getClientCnxnCount(ia);
+ if (maxClientCnxns > 0 && cnxncount >= maxClientCnxns){
+ LOG.warn("Too many connections from " + ia
+ " - max is " + maxClientCnxns );
- sc.close();
+ sc.close();
} else {
sc.configureBlocking(false);
SelectionKey sk = sc.register(selector,
@@ -218,7 +219,7 @@
NIOServerCnxn cnxn = createConnection(sc, sk);
sk.attach(cnxn);
addCnxn(cnxn);
- }
+ }
} else if ((k.readyOps() & (SelectionKey.OP_READ | SelectionKey.OP_WRITE)) != 0) {
NIOServerCnxn c = (NIOServerCnxn) k.attachment();
c.doIO(k);
@@ -288,8 +289,8 @@
}
- private void closeSessionWithoutWakeup(long sessionId) {
- synchronized (cnxns) {
+ private void closeSessionWithoutWakeup(long sessionId) {
+ synchronized (cnxns) {
for (Iterator<NIOServerCnxn> it = cnxns.iterator(); it
.hasNext();) {
NIOServerCnxn cnxn = it.next();
@@ -609,7 +610,7 @@
// session is setup
disableRecv();
if (connReq.getSessionId() != 0) {
- factory.closeSessionWithoutWakeup(connReq.getSessionId());
+ factory.closeSessionWithoutWakeup(connReq.getSessionId());
setSessionId(connReq.getSessionId());
zk.reopenSession(this, sessionId, passwd, sessionTimeout);
LOG.info("Renewing session 0x" + Long.toHexString(sessionId));
@@ -816,7 +817,7 @@
LOG.warn("Failed to unregister with JMX", e);
}
jmxConnectionBean = null;
-
+
if (closed) {
return;
}
@@ -824,7 +825,7 @@
synchronized (factory.ipMap)
{
Set<NIOServerCnxn> s = factory.ipMap.get(sock.socket().getInetAddress());
- s.remove(this);
+ s.remove(this);
}
synchronized (factory.cnxns) {
factory.cnxns.remove(this);
Added: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/PortAssignment.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/PortAssignment.java?rev=794887&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/PortAssignment.java (added)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/PortAssignment.java Thu Jul 16 22:09:40 2009
@@ -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;
+
+import org.apache.log4j.Logger;
+
+/** Assign ports to tests */
+public class PortAssignment {
+ private static final Logger LOG = Logger.getLogger(PortAssignment.class);
+
+ private static int nextPort = 11221;
+
+ /** Assign a new, unique port to the test */
+ public synchronized static int unique() {
+ LOG.info("assigning port " + nextPort);
+ return nextPort++;
+ }
+}
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/CRCTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/CRCTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/CRCTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/CRCTest.java Thu Jul 16 22:09:40 2009
@@ -39,6 +39,7 @@
import org.apache.jute.InputArchive;
import org.apache.log4j.Logger;
import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooKeeper;
@@ -48,14 +49,15 @@
import org.apache.zookeeper.server.persistence.FileTxnLog;
import org.apache.zookeeper.server.persistence.TxnLog.TxnIterator;
import org.apache.zookeeper.test.ClientBase;
+import org.junit.Test;
public class CRCTest extends TestCase implements Watcher{
-
private static final Logger LOG = Logger.getLogger(CRCTest.class);
- private static String HOSTPORT = "127.0.0.1:2357";
- ZooKeeperServer zks;
- private CountDownLatch startSignal;
-
+
+ private static final String HOSTPORT =
+ "127.0.0.1:" + PortAssignment.unique();
+ private volatile CountDownLatch startSignal;
+
@Override
protected void setUp() throws Exception {
LOG.info("STARTING " + getName());
@@ -64,7 +66,7 @@
protected void tearDown() throws Exception {
LOG.info("FINISHED " + getName());
}
-
+
/**
* corrupt a file by writing m at 500 b
* offset
@@ -77,7 +79,7 @@
byte[] b = "mahadev".getBytes();
long writeLen = 500L;
raf.seek(writeLen);
- //corruptting the data
+ //corrupting the data
raf.write(b);
raf.close();
}
@@ -108,28 +110,33 @@
crcIn.close();
return (val != checksum);
}
-
+
/** test checksums for the logs and snapshots.
- * the reader should fail on reading
+ * the reader should fail on reading
* a corrupt snapshot and a corrupt log
* file
* @throws Exception
*/
- public void testChecksums() throws Exception {
+ @Test
+ public void testChecksums() throws Exception {
File tmpDir = ClientBase.createTmpDir();
ClientBase.setupTestEnv();
- zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+ ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
SyncRequestProcessor.setSnapCount(150);
final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
f.startup(zks);
LOG.info("starting up the zookeeper server .. waiting");
- assertTrue("waiting for server being up",
+ assertTrue("waiting for server being up",
ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
- for (int i =0; i < 2000; i++) {
- zk.create("/crctest- " + i , ("/crctest- " + i).getBytes(),
- Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ try {
+ for (int i =0; i < 2000; i++) {
+ zk.create("/crctest- " + i , ("/crctest- " + i).getBytes(),
+ Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+ }
+ } finally {
+ zk.close();
}
f.shutdown();
assertTrue("waiting for server down",
@@ -138,7 +145,7 @@
File versionDir = new File(tmpDir, "version-2");
File[] list = versionDir.listFiles();
- //there should be only two files
+ //there should be only two files
// one the snapshot and the other logFile
File snapFile = null;
File logFile = null;
@@ -178,13 +185,13 @@
}
assertTrue(cfile);
}
-
+
public void process(WatchedEvent event) {
LOG.info("Event:" + event.getState() + " " + event.getType() + " " + event.getPath());
if (event.getState() == KeeperState.SyncConnected
&& startSignal != null && startSignal.getCount() > 0)
- {
- startSignal.countDown();
+ {
+ startSignal.countDown();
}
}
}
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DataTreeUnitTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DataTreeUnitTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DataTreeUnitTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DataTreeUnitTest.java Thu Jul 16 22:09:40 2009
@@ -20,14 +20,13 @@
import junit.framework.TestCase;
+import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.server.DataTree;
public class DataTreeUnitTest extends TestCase {
DataTree dt;
-
+
public void setUp() throws Exception {
dt=new DataTree();
}
@@ -36,7 +35,7 @@
dt=null;
}
-
+
public void testRootWatchTriggered() throws Exception {
class MyWatcher implements Watcher{
boolean fired=false;
@@ -44,7 +43,7 @@
if(event.getPath().equals("/"))
fired=true;
}
- };
+ }
MyWatcher watcher=new MyWatcher();
// set a watch on the root node
dt.getChildren("/", new Stat(), watcher);
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DeserializationPerfTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DeserializationPerfTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DeserializationPerfTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/DeserializationPerfTest.java Thu Jul 16 22:09:40 2009
@@ -26,9 +26,13 @@
import org.apache.jute.BinaryInputArchive;
import org.apache.jute.BinaryOutputArchive;
+import org.apache.log4j.Logger;
import org.apache.zookeeper.KeeperException;
+import org.junit.Test;
public class DeserializationPerfTest extends TestCase {
+ protected static final Logger LOG = Logger.getLogger(DeserializationPerfTest.class);
+
private static void deserializeTree(int depth, int width, int len)
throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
BinaryInputArchive ia;
@@ -58,46 +62,54 @@
assertEquals(count, dserTree.getNodeCount());
- System.out.println("Deserialized " + count + " nodes in " + durationms
+ LOG.info("Deserialized " + count + " nodes in " + durationms
+ " ms (" + pernodeus + "us/node), depth=" + depth + " width="
+ width + " datalen=" + len);
}
+ @Test
public void testSingleDeserialize() throws
InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
deserializeTree(1, 0, 20);
}
+ @Test
public void testWideDeserialize() throws
InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
deserializeTree(2, 50000, 20);
}
+ @Test
public void testDeepDeserialize() throws
InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
deserializeTree(1000, 1, 20);
}
+ @Test
public void test10Wide5DeepDeserialize() throws
InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
deserializeTree(5, 10, 20);
}
+ @Test
public void test15Wide5DeepDeserialize() throws
InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
deserializeTree(5, 15, 20);
}
+ @Test
public void test25Wide4DeepDeserialize() throws
InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
deserializeTree(4, 25, 20);
}
+ @Test
public void test40Wide4DeepDeserialize() throws
InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
deserializeTree(4, 40, 20);
}
+ @Test
public void test300Wide3DeepDeserialize() throws
InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
deserializeTree(3, 300, 20);
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/InvalidSnapshotTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/InvalidSnapshotTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/InvalidSnapshotTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/InvalidSnapshotTest.java Thu Jul 16 22:09:40 2009
@@ -25,43 +25,52 @@
import org.apache.log4j.Logger;
import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.test.ClientBase;
+import org.junit.Test;
/**
- * this test checks that the server works
+ * this test checks that the server works
* even if the last snapshot is invalidated
- * by corruption or if the server crashes
+ * by corruption or if the server crashes
* while generating the snapshot.
*/
public class InvalidSnapshotTest extends TestCase implements Watcher {
- private static final Logger LOG = Logger.getLogger(InvalidSnapshotTest.class);
- private static String HOSTPORT = "127.0.0.1:2357";
- ZooKeeperServer zks = null;
+ private static final Logger LOG =
+ Logger.getLogger(InvalidSnapshotTest.class);
+
+ private static final String HOSTPORT =
+ "127.0.0.1:" + PortAssignment.unique();
private static final int CONNECTION_TIMEOUT = 3000;
-
+
/**
- * this test does the main work of testing
+ * this test does the main work of testing
* an invalid snapshot
* @throws Exception
*/
+ @Test
public void testInvalidSnapshot() throws Exception {
File tmpDir = ClientBase.createTmpDir();
ClientBase.setupTestEnv();
- zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+ ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
SyncRequestProcessor.setSnapCount(100);
final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
f.startup(zks);
- assertTrue("waiting for server being up ",
+ assertTrue("waiting for server being up ",
ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
- for (int i=0; i< 2000; i++) {
- zk.create("/invalidsnap-" + i, new byte[0], Ids.OPEN_ACL_UNSAFE,
- CreateMode.PERSISTENT);
+ try {
+ for (int i=0; i< 2000; i++) {
+ zk.create("/invalidsnap-" + i, new byte[0], Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+ }
+ } finally {
+ zk.close();
}
f.shutdown();
assertTrue("waiting for server to shutdown",
@@ -76,19 +85,27 @@
SyncRequestProcessor.setSnapCount(100);
f = new NIOServerCnxn.Factory(PORT);
f.startup(zks);
- assertTrue("waiting for server being up ",
+ assertTrue("waiting for server being up ",
ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
// the server should come up
zk = new ZooKeeper(HOSTPORT, 20000, this);
- assertTrue("the node should exist", (zk.exists("/invalidsnap-1999", false) != null));
+ try {
+ assertTrue("the node should exist",
+ (zk.exists("/invalidsnap-1999", false) != null));
+ f.shutdown();
+ assertTrue("waiting for server to shutdown",
+ ClientBase.waitForServerDown(HOSTPORT, CONNECTION_TIMEOUT));
+ } finally {
+ zk.close();
+ }
+
f.shutdown();
assertTrue("waiting for server to shutdown",
ClientBase.waitForServerDown(HOSTPORT, CONNECTION_TIMEOUT));
-
}
-
+
public void process(WatchedEvent event) {
// do nothing for now
}
-
+
}
\ No newline at end of file
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/SerializationPerfTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/SerializationPerfTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/SerializationPerfTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/SerializationPerfTest.java Thu Jul 16 22:09:40 2009
@@ -24,9 +24,13 @@
import junit.framework.TestCase;
import org.apache.jute.BinaryOutputArchive;
+import org.apache.log4j.Logger;
import org.apache.zookeeper.KeeperException;
+import org.junit.Test;
public class SerializationPerfTest extends TestCase {
+ protected static final Logger LOG = Logger.getLogger(SerializationPerfTest.class);
+
private static class NullOutputStream extends OutputStream {
public void write(int b) {
// do nothing - exclude persistence from perf
@@ -66,46 +70,54 @@
long end = System.nanoTime();
long durationms = (end - start)/1000000L;
long pernodeus = ((end - start)/1000L)/count;
- System.out.println("Serialized " + count + " nodes in "
+ LOG.info("Serialized " + count + " nodes in "
+ durationms + " ms (" + pernodeus + "us/node), depth="
+ depth + " width=" + width + " datalen=" + len);
}
+ @Test
public void testSingleSerialize()
throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
serializeTree(1, 0, 20);
}
+ @Test
public void testWideSerialize()
throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
serializeTree(2, 50000, 20);
}
+ @Test
public void testDeepSerialize()
throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
serializeTree(1000, 1, 20);
}
+ @Test
public void test10Wide5DeepSerialize()
throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
serializeTree(5, 10, 20);
}
+ @Test
public void test15Wide5DeepSerialize()
throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
serializeTree(5, 15, 20);
}
+ @Test
public void test25Wide4DeepSerialize()
throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
serializeTree(4, 25, 20);
}
+ @Test
public void test40Wide4DeepSerialize()
throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
serializeTree(4, 40, 20);
}
+ @Test
public void test300Wide3DeepSerialize()
throws InterruptedException, IOException, KeeperException.NodeExistsException, KeeperException.NoNodeException {
serializeTree(3, 300, 20);
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/ZooKeeperServerTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/ZooKeeperServerTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/ZooKeeperServerTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/ZooKeeperServerTest.java Thu Jul 16 22:09:40 2009
@@ -21,12 +21,14 @@
import java.io.File;
import java.util.List;
+import junit.framework.TestCase;
+
import org.apache.zookeeper.server.persistence.FileTxnLog;
import org.apache.zookeeper.server.persistence.Util;
-
-import junit.framework.TestCase;
+import org.junit.Test;
public class ZooKeeperServerTest extends TestCase {
+ @Test
public void testSortDataDirAscending() {
File[] files = new File[5];
@@ -47,6 +49,7 @@
assertEquals(orig[4], filelist.get(4));
}
+ @Test
public void testSortDataDirDescending() {
File[] files = new File[5];
@@ -67,6 +70,7 @@
assertEquals(orig[2], filelist.get(4));
}
+ @Test
public void testGetLogFiles() {
File[] files = new File[5];
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java Thu Jul 16 22:09:40 2009
@@ -202,13 +202,13 @@
5000);
assertFalse("Server never came up", isup);
-
+
q1.shutdown();
assertTrue("waiting for server 1 down",
ClientBase.waitForServerDown("localhost:" + CLIENT_PORT_QP1,
ClientBase.CONNECTION_TIMEOUT));
-
+
} finally {
Logger.getLogger(org.apache.zookeeper.server.quorum.QuorumPeer.class)
.removeAppender(appender);
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ACLTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ACLTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ACLTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ACLTest.java Thu Jul 16 22:09:40 2009
@@ -29,6 +29,7 @@
import org.apache.log4j.Logger;
import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooKeeper;
@@ -42,9 +43,9 @@
public class ACLTest extends TestCase implements Watcher {
private static final Logger LOG = Logger.getLogger(ACLTest.class);
- private static String HOSTPORT = "127.0.0.1:2355";
- ZooKeeperServer zks;
- private CountDownLatch startSignal;
+ private static final String HOSTPORT =
+ "127.0.0.1:" + PortAssignment.unique();
+ private volatile CountDownLatch startSignal;
@Override
protected void setUp() throws Exception {
@@ -58,14 +59,14 @@
public void testDisconnectedAddAuth() throws Exception {
File tmpDir = ClientBase.createTmpDir();
ClientBase.setupTestEnv();
- zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+ ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
SyncRequestProcessor.setSnapCount(1000);
final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
f.startup(zks);
LOG.info("starting up the zookeeper server .. waiting");
- assertTrue("waiting for server being up",
- ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
+ assertTrue("waiting for server being up",
+ ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
try {
zk.addAuthInfo("digest", "pat:test".getBytes());
@@ -80,7 +81,7 @@
ClientBase.waitForServerDown(HOSTPORT,
ClientBase.CONNECTION_TIMEOUT));
}
-
+
/**
* Verify that acl optimization of storing just
* a few acls and there references in the data
@@ -89,14 +90,14 @@
public void testAcls() throws Exception {
File tmpDir = ClientBase.createTmpDir();
ClientBase.setupTestEnv();
- zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+ ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
SyncRequestProcessor.setSnapCount(1000);
final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
f.startup(zks);
LOG.info("starting up the zookeeper server .. waiting");
assertTrue("waiting for server being up",
- ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
+ ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
String path;
LOG.info("starting creating acls");
@@ -122,8 +123,7 @@
// now shutdown the server and restart it
f.shutdown();
assertTrue("waiting for server down",
- ClientBase.waitForServerDown(HOSTPORT,
- CONNECTION_TIMEOUT));
+ ClientBase.waitForServerDown(HOSTPORT, CONNECTION_TIMEOUT));
startSignal = new CountDownLatch(1);
zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
@@ -132,15 +132,14 @@
f.startup(zks);
assertTrue("waiting for server up",
- ClientBase.waitForServerUp(HOSTPORT,
- CONNECTION_TIMEOUT));
+ ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
startSignal.await(CONNECTION_TIMEOUT,
TimeUnit.MILLISECONDS);
assertTrue("count == 0", startSignal.getCount() == 0);
assertTrue("acl map ", (101 == zks.dataTree.longKeyMap.size()));
- for (int j =200; j < 205; j++) {
+ for (int j = 200; j < 205; j++) {
path = "/" + j;
ACL acl = new ACL();
acl.setPerms(0);
@@ -154,6 +153,8 @@
}
assertTrue("acl map ", (106 == zks.dataTree.longKeyMap.size()));
+ zk.close();
+
f.shutdown();
assertTrue("waiting for server down",
@@ -168,11 +169,15 @@
* @see org.apache.zookeeper.Watcher#process(org.apache.zookeeper.WatcherEvent)
*/
public void process(WatchedEvent event) {
- LOG.info("Event:" + event.getState() + " " + event.getType() + " " + event.getPath());
- if (event.getState() == KeeperState.SyncConnected
- && startSignal != null && startSignal.getCount() > 0)
- {
- startSignal.countDown();
+ LOG.info("Event:" + event.getState() + " " + event.getType() + " "
+ + event.getPath());
+ if (event.getState() == KeeperState.SyncConnected) {
+ if (startSignal != null && startSignal.getCount() > 0) {
+ LOG.info("startsignal.countDown()");
+ startSignal.countDown();
+ } else {
+ LOG.warn("startsignal " + startSignal);
+ }
}
}
}
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncTest.java Thu Jul 16 22:09:40 2009
@@ -190,6 +190,7 @@
}
// before restart
QuorumBase qt = new QuorumBase();
+ qt.setUp();
qt.verifyRootOfAllServersMatch(qb.hostPort);
tearDown();
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootClientTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootClientTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootClientTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootClientTest.java Thu Jul 16 22:09:40 2009
@@ -33,6 +33,7 @@
String hp = hostPort;
hostPort = hostPort + "/chrootclienttest";
+ System.out.println(hostPort);
super.setUp();
LOG.info("STARTING " + getName());
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ChrootTest.java Thu Jul 16 22:09:40 2009
@@ -27,16 +27,8 @@
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.test.AsyncOps.ACLCB;
-import org.apache.zookeeper.test.AsyncOps.ChildrenCB;
-import org.apache.zookeeper.test.AsyncOps.DataCB;
-import org.apache.zookeeper.test.AsyncOps.StatCB;
-import org.apache.zookeeper.test.AsyncOps.StringCB;
-import org.apache.zookeeper.test.AsyncOps.VoidCB;
-import org.junit.Test;
public class ChrootTest extends ClientBase {
private class MyWatcher implements Watcher {
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java Thu Jul 16 22:09:40 2009
@@ -25,6 +25,7 @@
import java.io.OutputStream;
import java.net.Socket;
import java.util.Arrays;
+import java.util.LinkedList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
@@ -35,6 +36,7 @@
import org.apache.log4j.Logger;
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;
@@ -51,7 +53,7 @@
static final File BASETEST =
new File(System.getProperty("build.test.dir", "build"));
- protected String hostPort = "127.0.0.1:33221";
+ protected String hostPort = "127.0.0.1:" + PortAssignment.unique();
protected int maxCnxns = 0;
protected NIOServerCnxn.Factory serverFactory = null;
protected File tmpDir = null;
@@ -59,10 +61,6 @@
super();
}
- public ClientBase(String name) {
- super(name);
- }
-
/**
* 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
@@ -137,6 +135,9 @@
return createClient(watcher, hp);
}
+ private LinkedList<ZooKeeper> allClients;
+ private boolean allClientsSetup = false;
+
protected TestableZooKeeper createClient(CountdownWatcher watcher, String hp)
throws IOException, InterruptedException
{
@@ -148,6 +149,19 @@
{
fail("Unable to connect to server");
}
+ synchronized(this) {
+ if (!allClientsSetup) {
+ LOG.error("allClients never setup");
+ fail("allClients never setup");
+ }
+ if (allClients != null) {
+ allClients.add(zk);
+ } else {
+ // test done - close the zk, not needed
+ zk.close();
+ }
+ }
+
JMXEnv.ensureAll("0x" + Long.toHexString(zk.getSessionId()));
return zk;
@@ -247,11 +261,11 @@
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");
assertFalse(tmpDir.exists()); // never true if tmpfile does it's job
assertTrue(tmpDir.mkdirs());
-
+
return tmpDir;
}
private static int getPort(String hostPort) {
@@ -262,10 +276,10 @@
}
return Integer.parseInt(portstr);
}
-
+
static NIOServerCnxn.Factory createNewServerInstance(File dataDir,
NIOServerCnxn.Factory factory, String hostPort, int maxCnxns)
- throws IOException, InterruptedException
+ throws IOException, InterruptedException
{
ZooKeeperServer zks = new ZooKeeperServer(dataDir, dataDir, 3000);
final int PORT = getPort(hostPort);
@@ -306,6 +320,11 @@
FileTxnLog.setPreallocSize(100 * 1024);
}
+ protected void setUpAll() throws Exception {
+ allClients = new LinkedList<ZooKeeper>();
+ allClientsSetup = true;
+ }
+
@Override
protected void setUp() throws Exception {
LOG.info("STARTING " + getName());
@@ -313,6 +332,8 @@
JMXEnv.setUp();
+ setUpAll();
+
tmpDir = createTmpDir(BASETEST);
startServer();
@@ -334,19 +355,37 @@
// ensure no beans are leftover
JMXEnv.ensureOnly();
}
-
+
+ protected void tearDownAll() throws Exception {
+ synchronized (this) {
+ for (ZooKeeper zk : allClients) {
+ try {
+ if (zk != null)
+ zk.close();
+ } catch (InterruptedException e) {
+ LOG.warn("ignoring interrupt", e);
+ }
+ }
+ allClients = null;
+ }
+ }
+
@Override
protected void tearDown() throws Exception {
LOG.info("tearDown starting");
+ tearDownAll();
stopServer();
-
+
if (tmpDir != null) {
//assertTrue("delete " + tmpDir.toString(), recursiveDelete(tmpDir));
// FIXME see ZOOKEEPER-121 replace following line with previous
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();
LOG.info("FINISHED " + getName());
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientRetry.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientRetry.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientRetry.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientRetry.java Thu Jul 16 22:09:40 2009
@@ -19,10 +19,11 @@
import java.io.IOException;
import java.util.concurrent.TimeoutException;
+
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.ZooKeeper.States;
-public class ClientRetry extends ClientBase{
+public class ClientRetry extends ClientBase {
public void setUp() throws Exception {
maxCnxns = 1;
@@ -44,17 +45,24 @@
CountdownWatcher cdw1 = new CountdownWatcher();
CountdownWatcher cdw2 = new CountdownWatcher();
ZooKeeper zk = new ZooKeeper(hostPort, 10000, cdw1);
- cdw1.waitForConnected(CONNECTION_TIMEOUT);
- ZooKeeper zk2 = new ZooKeeper(hostPort, 10000, cdw2);
- States s1 = zk.getState();
- States s2 = zk2.getState();
- assertSame(s1,States.CONNECTED);
- assertSame(s2,States.CONNECTING);
- cdw1.reset();
- zk.close();
- cdw1.waitForDisconnected(CONNECTION_TIMEOUT);
- cdw2.waitForConnected(CONNECTION_TIMEOUT);
- assertSame(zk2.getState(),States.CONNECTED);
+ try {
+ cdw1.waitForConnected(CONNECTION_TIMEOUT);
+ ZooKeeper zk2 = new ZooKeeper(hostPort, 10000, cdw2);
+ try {
+ States s1 = zk.getState();
+ States s2 = zk2.getState();
+ assertSame(s1,States.CONNECTED);
+ assertSame(s2,States.CONNECTING);
+ cdw1.reset();
+ cdw1.waitForDisconnected(CONNECTION_TIMEOUT);
+ cdw2.waitForConnected(CONNECTION_TIMEOUT);
+ assertSame(zk2.getState(),States.CONNECTED);
+ } finally {
+ zk2.close();
+ }
+ } finally {
+ zk.close();
+ }
}
}
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/DataTreeTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/DataTreeTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/DataTreeTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/DataTreeTest.java Thu Jul 16 22:09:40 2009
@@ -21,10 +21,11 @@
import junit.framework.TestCase;
import org.apache.log4j.Logger;
+import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.server.DataTree;
+import org.junit.Test;
public class DataTreeTest extends TestCase {
protected static final Logger LOG = Logger.getLogger(DataTreeTest.class);
@@ -43,6 +44,7 @@
LOG.info("FINISHED " + getName());
}
+ @Test
public void testRootWatchTriggered() throws Exception {
class MyWatcher implements Watcher{
boolean fired=false;
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/EventTypeTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/EventTypeTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/EventTypeTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/EventTypeTest.java Thu Jul 16 22:09:40 2009
@@ -21,9 +21,9 @@
import java.util.EnumSet;
import junit.framework.TestCase;
-import org.junit.Test;
import org.apache.zookeeper.Watcher.Event.EventType;
+import org.junit.Test;
public class EventTypeTest extends TestCase {
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLENewEpochTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLENewEpochTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLENewEpochTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLENewEpochTest.java Thu Jul 16 22:09:40 2009
@@ -27,6 +27,7 @@
import junit.framework.TestCase;
import org.apache.log4j.Logger;
+import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.server.quorum.FastLeaderElection;
import org.apache.zookeeper.server.quorum.QuorumPeer;
import org.apache.zookeeper.server.quorum.Vote;
@@ -38,8 +39,6 @@
protected static final Logger LOG = Logger.getLogger(FLENewEpochTest.class);
int count;
- int baseport;
- int baseLEport;
HashMap<Long,QuorumServer> peers;
ArrayList<LEThread> threads;
File tmpdir[];
@@ -52,8 +51,6 @@
@Override
public void setUp() throws Exception {
count = 3;
- baseport= 33303;
- baseLEport = 43303;
peers = new HashMap<Long,QuorumServer>(count);
threads = new ArrayList<LEThread>(count);
@@ -101,7 +98,7 @@
LOG.info("Going to call leader election again: " + i);
v = peer.getElectionAlg().lookForLeader();
- if(v == null){
+ if (v == null){
fail("Thread " + i + " got a null vote");
}
@@ -114,7 +111,7 @@
LOG.info("Finished election: " + i + ", " + v.id);
//votes[i] = v;
- switch(i){
+ switch (i) {
case 0:
LOG.info("First peer, do nothing, just join");
if(finish0.tryAcquire(1000, java.util.concurrent.TimeUnit.MILLISECONDS)){
@@ -128,8 +125,7 @@
if(round[1] != 0){
finish0.release();
flag = false;
- }
- else{
+ } else {
finish3.acquire();
start0.release();
}
@@ -147,7 +143,7 @@
break;
}
}
- } catch (Exception e){
+ } catch (Exception e) {
e.printStackTrace();
}
}
@@ -161,10 +157,12 @@
LOG.info("TestLE: " + getName()+ ", " + count);
for(int i = 0; i < count; i++) {
- peers.put(Long.valueOf(i), new QuorumServer(i, new InetSocketAddress(baseport+100+i),
- new InetSocketAddress(baseLEport+100+i)));
+ peers.put(Long.valueOf(i),
+ new QuorumServer(i,
+ new InetSocketAddress(PortAssignment.unique()),
+ new InetSocketAddress(PortAssignment.unique())));
tmpdir[i] = ClientBase.createTmpDir();
- port[i] = baseport+i;
+ port[i] = PortAssignment.unique();
}
for(int i = 1; i < le.length; i++) {
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLERestartTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLERestartTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLERestartTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLERestartTest.java Thu Jul 16 22:09:40 2009
@@ -26,10 +26,10 @@
import java.util.Random;
import java.util.concurrent.Semaphore;
-
import junit.framework.TestCase;
import org.apache.log4j.Logger;
+import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.server.quorum.FastLeaderElection;
import org.apache.zookeeper.server.quorum.QuorumPeer;
import org.apache.zookeeper.server.quorum.Vote;
@@ -58,8 +58,8 @@
}
int count;
- int baseport;
- int baseLEport;
+ // int baseport;
+ // int baseLEport;
HashMap<Long,QuorumServer> peers;
ArrayList<FLERestartThread> restartThreads;
HashMap<Integer, HashSet<TestVote> > voteMap;
@@ -67,7 +67,7 @@
int port[];
int successCount;
Semaphore finish;
-
+
volatile Vote votes[];
volatile boolean leaderDies;
volatile long leader = -1;
@@ -77,8 +77,6 @@
@Override
public void setUp() throws Exception {
count = 3;
- baseport= 33003;
- baseLEport = 43003;
peers = new HashMap<Long,QuorumServer>(count);
restartThreads = new ArrayList<FLERestartThread>(count);
@@ -137,23 +135,23 @@
LOG.info("First peer, shutting it down");
peer.shutdown();
((FastLeaderElection) restartThreads.get(i).peer.getElectionAlg()).shutdown();
-
+
peer = new QuorumPeer(peers, tmpdir[i], tmpdir[i], port[i], 3, i, 2, 2, 2);
peer.startLeaderElection();
peerRound++;
- } else {
+ } else {
finish.release(2);
return;
- }
-
+ }
+
break;
case 1:
LOG.info("Second entering case");
finish.acquire();
//if(threads.get(0).peer.getPeerState() == ServerState.LEADING ){
LOG.info("Release");
-
- return;
+
+ return;
case 2:
LOG.info("First peer, do nothing, just join");
finish.acquire();
@@ -168,8 +166,8 @@
}
}
}
-
-
+
+
@Test
public void testLERestart() throws Exception {
@@ -179,10 +177,12 @@
LOG.info("TestLE: " + getName()+ ", " + count);
for(int i = 0; i < count; i++) {
- peers.put(Long.valueOf(i), new QuorumServer(i, new InetSocketAddress(baseport+100+i),
- new InetSocketAddress(baseLEport+100+i)));
+ peers.put(Long.valueOf(i),
+ new QuorumServer(i,
+ new InetSocketAddress(PortAssignment.unique()),
+ new InetSocketAddress(PortAssignment.unique())));
tmpdir[i] = ClientBase.createTmpDir();
- port[i] = baseport+i;
+ port[i] = PortAssignment.unique();
}
for(int i = 0; i < count; i++) {
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLETest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLETest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLETest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/FLETest.java Thu Jul 16 22:09:40 2009
@@ -28,6 +28,7 @@
import junit.framework.TestCase;
import org.apache.log4j.Logger;
+import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.server.quorum.FastLeaderElection;
import org.apache.zookeeper.server.quorum.QuorumPeer;
import org.apache.zookeeper.server.quorum.Vote;
@@ -37,6 +38,7 @@
public class FLETest extends TestCase {
protected static final Logger LOG = Logger.getLogger(FLETest.class);
+ private FLETest.LEThread leThread;
static class TestVote {
TestVote(int id, long leader) {
@@ -56,8 +58,6 @@
}
int count;
- int baseport;
- int baseLEport;
HashMap<Long,QuorumServer> peers;
ArrayList<LEThread> threads;
HashMap<Integer, HashSet<TestVote> > voteMap;
@@ -75,8 +75,6 @@
@Override
public void setUp() throws Exception {
count = 7;
- baseport= 33003;
- baseLEport = 43003;
peers = new HashMap<Long,QuorumServer>(count);
threads = new ArrayList<LEThread>(count);
@@ -92,8 +90,10 @@
@Override
public void tearDown() throws Exception {
- for(int i = 0; i < threads.size(); i++) {
- ((FastLeaderElection) threads.get(i).peer.getElectionAlg()).shutdown();
+ for (int i = 0; i < threads.size(); i++) {
+ leThread = threads.get(i);
+ ((FastLeaderElection) leThread.peer.getElectionAlg()).shutdown();
+ leThread.peer.shutdown();
}
LOG.info("FINISHED " + getName());
}
@@ -264,14 +264,17 @@
LOG.info("TestLE: " + getName()+ ", " + count);
for(int i = 0; i < count; i++) {
- peers.put(Long.valueOf(i), new QuorumServer(i, new InetSocketAddress(baseport+100+i),
- new InetSocketAddress(baseLEport+100+i)));
+ peers.put(Long.valueOf(i),
+ new QuorumServer(i,
+ new InetSocketAddress(PortAssignment.unique()),
+ new InetSocketAddress(PortAssignment.unique())));
tmpdir[i] = ClientBase.createTmpDir();
- port[i] = baseport+i;
+ port[i] = PortAssignment.unique();
}
for(int i = 0; i < le.length; i++) {
- QuorumPeer peer = new QuorumPeer(peers, tmpdir[i], tmpdir[i], port[i], 3, i, 2, 2, 2);
+ QuorumPeer peer = new QuorumPeer(peers, tmpdir[i], tmpdir[i],
+ port[i], 3, i, 2, 2, 2);
peer.startLeaderElection();
LEThread thread = new LEThread(peer, i);
thread.start();
@@ -288,12 +291,13 @@
}
}
- /*
- * Lists what threads haven-t joined. A thread doesn't join if it hasn't decided
- * upon a leader yet. It can happen that a peer is slow or disconnected, and it can
- * take longer to nominate and connect to the current leader.
+ /*
+ * Lists what threads haven-t joined. A thread doesn't join if
+ * it hasn't decided upon a leader yet. It can happen that a
+ * peer is slow or disconnected, and it can take longer to
+ * nominate and connect to the current leader.
*/
- for(int i = 0; i < threads.size(); i++) {
+ for (int i = 0; i < threads.size(); i++) {
if (threads.get(i).isAlive()) {
LOG.info("Threads didn't join: " + i);
}
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/HierarchicalQuorumTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/HierarchicalQuorumTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/HierarchicalQuorumTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/HierarchicalQuorumTest.java Thu Jul 16 22:09:40 2009
@@ -28,6 +28,7 @@
import junit.framework.TestCase;
import org.apache.log4j.Logger;
+import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.server.quorum.FastLeaderElection;
import org.apache.zookeeper.server.quorum.QuorumPeer;
import org.apache.zookeeper.server.quorum.Vote;
@@ -43,8 +44,6 @@
Properties qp;
int count;
- int baseport;
- int baseLEport;
HashMap<Long,QuorumServer> peers;
ArrayList<LEThread> threads;
File tmpdir[];
@@ -61,8 +60,6 @@
@Override
protected void setUp() throws Exception {
count = 9;
- baseport= 33003;
- baseLEport = 43003;
peers = new HashMap<Long,QuorumServer>(count);
threads = new ArrayList<LEThread>(count);
@@ -93,7 +90,12 @@
protected void tearDown() throws Exception {
for(int i = 0; i < threads.size(); i++) {
- ((FastLeaderElection) threads.get(i).peer.getElectionAlg()).shutdown();
+ LEThread leThread = threads.get(i);
+ ((FastLeaderElection) leThread.peer.getElectionAlg()).shutdown();
+ // shutdown() has to be explicitly called for every thread to
+ // make sure that resources are freed properly and all fixed network ports
+ // are available for other test cases
+ leThread.peer.shutdown();
}
LOG.info("FINISHED " + getName());
}
@@ -148,10 +150,12 @@
LOG.info("TestHierarchicalQuorum: " + getName()+ ", " + count);
for(int i = 0; i < count; i++) {
- peers.put(Long.valueOf(i), new QuorumServer(i, new InetSocketAddress(baseport+100+i),
- new InetSocketAddress(baseLEport+100+i)));
+ peers.put(Long.valueOf(i),
+ new QuorumServer(i,
+ new InetSocketAddress(PortAssignment.unique()),
+ new InetSocketAddress(PortAssignment.unique())));
tmpdir[i] = ClientBase.createTmpDir();
- port[i] = baseport+i;
+ port[i] = PortAssignment.unique();
}
for(int i = 0; i < le.length; i++) {
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/IntegrityCheck.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/IntegrityCheck.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/IntegrityCheck.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/IntegrityCheck.java Thu Jul 16 22:09:40 2009
@@ -36,17 +36,16 @@
import java.util.HashMap;
import org.apache.log4j.Logger;
-
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.Watcher.Event.KeeperState;
import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.AsyncCallback.DataCallback;
import org.apache.zookeeper.AsyncCallback.StatCallback;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.WatchedEvent;
public class IntegrityCheck implements Watcher, StatCallback, DataCallback {
private static final Logger LOG = Logger.getLogger(IntegrityCheck.class);
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/InvalidSnapshotTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/InvalidSnapshotTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/InvalidSnapshotTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/InvalidSnapshotTest.java Thu Jul 16 22:09:40 2009
@@ -26,6 +26,7 @@
import junit.framework.TestCase;
import org.apache.log4j.Logger;
+import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooKeeper;
@@ -36,12 +37,13 @@
public class InvalidSnapshotTest extends TestCase implements Watcher {
private final static Logger LOG = Logger.getLogger(UpgradeTest.class);
- private static String HOSTPORT = "127.0.0.1:2359";
- ZooKeeperServer zks;
+ private static final String HOSTPORT =
+ "127.0.0.1:" + PortAssignment.unique();
+
private static final File testData = new File(
System.getProperty("test.data.dir", "build/test/data"));
private CountDownLatch startSignal;
-
+
@Override
protected void setUp() throws Exception {
LOG.info("STARTING " + getName());
@@ -50,39 +52,43 @@
protected void tearDown() throws Exception {
LOG.info("FINISHED " + getName());
}
-
+
/**
* test the snapshot
- * @throws Exception
+ * @throws Exception an exception could be expected
*/
public void testSnapshot() throws Exception {
File snapDir = new File(testData, "invalidsnap");
- zks = new ZooKeeperServer(snapDir, snapDir, 3000);
+ ZooKeeperServer zks = new ZooKeeperServer(snapDir, snapDir, 3000);
SyncRequestProcessor.setSnapCount(1000);
final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
f.startup(zks);
LOG.info("starting up the zookeeper server .. waiting");
- assertTrue("waiting for server being up",
- ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
+ assertTrue("waiting for server being up",
+ ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
ZooKeeper zk = new ZooKeeper(HOSTPORT, 20000, this);
- // we know this from the data files
- // this node is the last node in the snapshot
-
- assertTrue(zk.exists("/9/9/8", false) != null);
+ try {
+ // we know this from the data files
+ // this node is the last node in the snapshot
+
+ assertTrue(zk.exists("/9/9/8", false) != null);
+ } finally {
+ zk.close();
+ }
f.shutdown();
assertTrue("waiting for server down",
ClientBase.waitForServerDown(HOSTPORT,
ClientBase.CONNECTION_TIMEOUT));
-
+
}
-
+
public void process(WatchedEvent event) {
LOG.info("Event:" + event.getState() + " " + event.getType() + " " + event.getPath());
if (event.getState() == KeeperState.SyncConnected
&& startSignal != null && startSignal.getCount() > 0)
- {
- startSignal.countDown();
+ {
+ startSignal.countDown();
}
}
}
\ No newline at end of file
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LETest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LETest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LETest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/LETest.java Thu Jul 16 22:09:40 2009
@@ -26,6 +26,7 @@
import junit.framework.TestCase;
+import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.server.quorum.LeaderElection;
import org.apache.zookeeper.server.quorum.QuorumPeer;
import org.apache.zookeeper.server.quorum.Vote;
@@ -84,22 +85,25 @@
}
public void testLE() throws Exception {
int count = 30;
- int baseport= 33003;
HashMap<Long,QuorumServer> peers = new HashMap<Long,QuorumServer>(count);
ArrayList<LEThread> threads = new ArrayList<LEThread>(count);
File tmpdir[] = new File[count];
int port[] = new int[count];
votes = new Vote[count];
for(int i = 0; i < count; i++) {
- peers.put(Long.valueOf(i), new QuorumServer(i, new InetSocketAddress("127.0.0.1", baseport+100+i)));
+ peers.put(Long.valueOf(i),
+ new QuorumServer(i,
+ new InetSocketAddress("127.0.0.1",
+ PortAssignment.unique())));
tmpdir[i] = ClientBase.createTmpDir();
- port[i] = baseport+i;
+ port[i] = PortAssignment.unique();
}
LeaderElection le[] = new LeaderElection[count];
leaderDies = true;
boolean allowOneBadLeader = leaderDies;
for(int i = 0; i < le.length; i++) {
- QuorumPeer peer = new QuorumPeer(peers, tmpdir[i], tmpdir[i], port[i], 0, i, 2, 2, 2);
+ QuorumPeer peer = new QuorumPeer(peers, tmpdir[i], tmpdir[i],
+ port[i], 0, i, 2, 2, 2);
peer.startLeaderElection();
le[i] = new LeaderElection(peer);
LEThread thread = new LEThread(le[i], peer, i);
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/NullDataTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/NullDataTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/NullDataTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/NullDataTest.java Thu Jul 16 22:09:40 2009
@@ -25,9 +25,9 @@
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.AsyncCallback.StatCallback;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.AsyncCallback.StatCallback;
public class NullDataTest extends ClientBase implements StatCallback {
String snapCount;
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/OOMTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/OOMTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/OOMTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/OOMTest.java Thu Jul 16 22:09:40 2009
@@ -26,20 +26,23 @@
import junit.framework.TestCase;
+import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.PortAssignment;
+import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.data.Stat;
-import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.server.NIOServerCnxn;
import org.apache.zookeeper.server.ZooKeeperServer;
+import org.junit.Test;
/**
*
*/
public class OOMTest extends TestCase implements Watcher {
+ @Test
public void testOOM() throws IOException, InterruptedException, KeeperException {
// This test takes too long to run!
if (true)
@@ -58,8 +61,8 @@
}
ClientBase.setupTestEnv();
ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
-
- final int PORT = 33221;
+
+ final int PORT = PortAssignment.unique();
NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
f.startup(zks);
assertTrue("waiting for server up",
@@ -67,14 +70,14 @@
CONNECTION_TIMEOUT));
System.err.println("OOM Stage 0");
- utestPrep();
+ utestPrep(PORT);
System.out.println("Free = " + Runtime.getRuntime().freeMemory()
+ " total = " + Runtime.getRuntime().totalMemory() + " max = "
+ Runtime.getRuntime().maxMemory());
System.err.println("OOM Stage 1");
for (int i = 0; i < 1000; i++) {
System.out.println(i);
- utestExists();
+ utestExists(PORT);
}
System.out.println("Free = " + Runtime.getRuntime().freeMemory()
+ " total = " + Runtime.getRuntime().totalMemory() + " max = "
@@ -82,7 +85,7 @@
System.err.println("OOM Stage 2");
for (int i = 0; i < 1000; i++) {
System.out.println(i);
- utestGet();
+ utestGet(PORT);
}
System.out.println("Free = " + Runtime.getRuntime().freeMemory()
+ " total = " + Runtime.getRuntime().totalMemory() + " max = "
@@ -90,7 +93,7 @@
System.err.println("OOM Stage 3");
for (int i = 0; i < 1000; i++) {
System.out.println(i);
- utestChildren();
+ utestChildren(PORT);
}
System.out.println("Free = " + Runtime.getRuntime().freeMemory()
+ " total = " + Runtime.getRuntime().totalMemory() + " max = "
@@ -103,28 +106,33 @@
CONNECTION_TIMEOUT));
}
- private void utestExists() throws IOException, InterruptedException, KeeperException {
+ private void utestExists(int port)
+ throws IOException, InterruptedException, KeeperException
+ {
ZooKeeper zk =
- new ZooKeeper("127.0.0.1:33221", CONNECTION_TIMEOUT, this);
+ new ZooKeeper("127.0.0.1:" + port, CONNECTION_TIMEOUT, this);
for (int i = 0; i < 10000; i++) {
zk.exists("/this/path/doesnt_exist!", true);
}
zk.close();
}
- private void utestPrep() throws IOException,
- InterruptedException, KeeperException {
+ private void utestPrep(int port)
+ throws IOException, InterruptedException, KeeperException
+ {
ZooKeeper zk =
- new ZooKeeper("127.0.0.1:33221", CONNECTION_TIMEOUT, this);
+ new ZooKeeper("127.0.0.1:" + port, CONNECTION_TIMEOUT, this);
for (int i = 0; i < 10000; i++) {
zk.create("/" + i, null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
}
zk.close();
}
- private void utestGet() throws IOException, InterruptedException, KeeperException {
+ private void utestGet(int port)
+ throws IOException, InterruptedException, KeeperException
+ {
ZooKeeper zk =
- new ZooKeeper("127.0.0.1:33221", CONNECTION_TIMEOUT, this);
+ new ZooKeeper("127.0.0.1:" + port, CONNECTION_TIMEOUT, this);
for (int i = 0; i < 10000; i++) {
Stat stat = new Stat();
zk.getData("/" + i, true, stat);
@@ -132,9 +140,11 @@
zk.close();
}
- private void utestChildren() throws IOException, InterruptedException, KeeperException {
+ private void utestChildren(int port)
+ throws IOException, InterruptedException, KeeperException
+ {
ZooKeeper zk =
- new ZooKeeper("127.0.0.1:33221", CONNECTION_TIMEOUT, this);
+ new ZooKeeper("127.0.0.1:" + port, CONNECTION_TIMEOUT, this);
for (int i = 0; i < 10000; i++) {
zk.getChildren("/" + i, true);
}
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/PurgeTxnTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/PurgeTxnTest.java?rev=794887&r1=794886&r2=794887&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/PurgeTxnTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/PurgeTxnTest.java Thu Jul 16 22:09:40 2009
@@ -23,8 +23,8 @@
import junit.framework.TestCase;
-import org.apache.log4j.Logger;
import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooKeeper;
@@ -35,33 +35,36 @@
import org.apache.zookeeper.server.ZooKeeperServer;
import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
-/**
+/**
* test the purging of the logs
* and purging of the snapshots.
*/
public class PurgeTxnTest extends TestCase implements Watcher {
- private static final Logger LOG = Logger.getLogger(PurgeTxnTest.class);
- private static String HOSTPORT = "127.0.0.1:2357";
- ZooKeeperServer zks = null;
+ //private static final Logger LOG = Logger.getLogger(PurgeTxnTest.class);
+ private static String HOSTPORT = "127.0.0.1:" + PortAssignment.unique();
private static final int CONNECTION_TIMEOUT = 3000;
/**
* test the purge
- * @throws Exception
+ * @throws Exception an exception might be thrown here
*/
public void testPurge() throws Exception {
File tmpDir = ClientBase.createTmpDir();
ClientBase.setupTestEnv();
- zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+ ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
SyncRequestProcessor.setSnapCount(100);
final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
NIOServerCnxn.Factory f = new NIOServerCnxn.Factory(PORT);
f.startup(zks);
- assertTrue("waiting for server being up ",
+ assertTrue("waiting for server being up ",
ClientBase.waitForServerUp(HOSTPORT,CONNECTION_TIMEOUT));
ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, this);
- for (int i=0; i< 2000; i++) {
- zk.create("/invalidsnap-" + i, new byte[0], Ids.OPEN_ACL_UNSAFE,
- CreateMode.PERSISTENT);
+ try {
+ for (int i = 0; i< 2000; i++) {
+ zk.create("/invalidsnap-" + i, new byte[0], Ids.OPEN_ACL_UNSAFE,
+ CreateMode.PERSISTENT);
+ }
+ } finally {
+ zk.close();
}
f.shutdown();
assertTrue("waiting for server to shutdown",
@@ -78,9 +81,9 @@
}
assertTrue("exactly 3 snapshots ", (numSnaps == 3));
}
-
+
public void process(WatchedEvent event) {
// do nothing
}
-
+
}
|