Author: mahadev
Date: Thu May 14 20:15:06 2009
New Revision: 774909
URL: http://svn.apache.org/viewvc?rev=774909&view=rev
Log:
ZOOKEEPER-94. JMX tests are needed to verify that the JMX MBeans work properly (phunt via
mahadev)
Added:
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/JMXEnv.java
Modified:
hadoop/zookeeper/trunk/CHANGES.txt
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/AsyncTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientBase.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java
hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/QuorumBase.java
Modified: hadoop/zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/CHANGES.txt?rev=774909&r1=774908&r2=774909&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/CHANGES.txt (original)
+++ hadoop/zookeeper/trunk/CHANGES.txt Thu May 14 20:15:06 2009
@@ -130,6 +130,9 @@
ZOOKEEPER-300. zk jmx code is calling printStackTrace when creating bean
name (should not be) (phunt via mahadev)
+
+ ZOOKEEPER-94. JMX tests are needed to verify that the JMX MBeans work
+properly (phunt via mahadev)
NEW FEATURES:
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=774909&r1=774908&r2=774909&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 May
14 20:15:06 2009
@@ -62,6 +62,7 @@
}
protected void restart() throws Exception {
+ JMXEnv.setUp();
qb.startServers();
}
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=774909&r1=774908&r2=774909&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 May
14 20:15:06 2009
@@ -29,6 +29,8 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
+import javax.management.MBeanServerConnection;
+
import junit.framework.TestCase;
import org.apache.log4j.Logger;
@@ -72,8 +74,16 @@
protected static class CountdownWatcher implements Watcher {
// XXX this doesn't need to be volatile! (Should probably be final)
- volatile CountDownLatch clientConnected = new CountDownLatch(1);
+ volatile CountDownLatch clientConnected;
volatile boolean connected;
+
+ public CountdownWatcher() {
+ reset();
+ }
+ synchronized public void reset() {
+ clientConnected = new CountDownLatch(1);
+ connected = false;
+ }
synchronized public void process(WatchedEvent event) {
if (event.getState() == KeeperState.SyncConnected) {
connected = true;
@@ -129,6 +139,7 @@
protected TestableZooKeeper createClient(CountdownWatcher watcher, String hp)
throws IOException, InterruptedException
{
+ watcher.reset();
TestableZooKeeper zk =
new TestableZooKeeper(hp, CONNECTION_TIMEOUT, watcher);
if (!watcher.clientConnected.await(CONNECTION_TIMEOUT,
@@ -136,6 +147,8 @@
{
fail("Unable to connect to server");
}
+ JMXEnv.ensureAll("0x" + Long.toHexString(zk.getSessionId()));
+
return zk;
}
@@ -288,41 +301,53 @@
protected void setUp() throws Exception {
LOG.info("STARTING " + getName());
+ JMXEnv.setUp();
+
tmpDir = createTmpDir(BASETEST);
setupTestEnv();
- serverFactory =
- createNewServerInstance(tmpDir, serverFactory, hostPort);
+
+ startServer();
LOG.info("Client test setup finished");
}
+ protected void startServer() throws Exception {
+ LOG.info("STARTING server");
+ serverFactory = createNewServerInstance(tmpDir, serverFactory, hostPort);
+ // ensure that only server and data bean are registered
+ JMXEnv.ensureOnly("InMemoryDataTree", "StandaloneServer_port");
+ }
+
protected void stopServer() throws Exception {
LOG.info("STOPPING server");
shutdownServerInstance(serverFactory, hostPort);
serverFactory = null;
- }
-
- protected void startServer() throws Exception {
- LOG.info("STARTING server");
- serverFactory = createNewServerInstance(tmpDir, serverFactory, hostPort);
+ // ensure no beans are leftover
+ JMXEnv.ensureOnly();
}
@Override
protected void tearDown() throws Exception {
LOG.info("tearDown starting");
- shutdownServerInstance(serverFactory, hostPort);
+ stopServer();
if (tmpDir != null) {
//assertTrue("delete " + tmpDir.toString(), recursiveDelete(tmpDir));
// FIXME see ZOOKEEPER-121 replace following line with previous
recursiveDelete(tmpDir);
}
+
+ JMXEnv.tearDown();
LOG.info("FINISHED " + getName());
}
+ public static MBeanServerConnection jmxConn() throws IOException {
+ return JMXEnv.conn();
+ }
+
private static boolean recursiveDelete(File d) {
if (d.isDirectory()) {
File children[] = d.listFiles();
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java?rev=774909&r1=774908&r2=774909&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ClientTest.java Thu May
14 20:15:06 2009
@@ -59,6 +59,7 @@
try {
zkIdle = createClient();
zkWatchCreator = createClient();
+
for (int i = 0; i < 30; i++) {
zkWatchCreator.create("/" + i, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
}
Added: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/JMXEnv.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/JMXEnv.java?rev=774909&view=auto
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/JMXEnv.java (added)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/JMXEnv.java Thu May 14
20:15:06 2009
@@ -0,0 +1,176 @@
+/**
+ * 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.lang.management.ManagementFactory;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+import javax.management.MBeanServer;
+import javax.management.MBeanServerConnection;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.remote.JMXConnector;
+import javax.management.remote.JMXConnectorFactory;
+import javax.management.remote.JMXConnectorServer;
+import javax.management.remote.JMXConnectorServerFactory;
+import javax.management.remote.JMXServiceURL;
+
+import junit.framework.TestCase;
+
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.jmx.CommonNames;
+
+public class JMXEnv {
+ protected static final Logger LOG = Logger.getLogger(JMXEnv.class);
+
+ private static JMXConnectorServer cs;
+ private static JMXConnector cc;
+
+ public static void setUp() throws IOException {
+ MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+
+ JMXServiceURL url = new JMXServiceURL("service:jmx:rmi://");
+ cs = JMXConnectorServerFactory.newJMXConnectorServer(url, null, mbs);
+ cs.start();
+
+ JMXServiceURL addr = cs.getAddress();
+
+ cc = JMXConnectorFactory.connect(addr);
+ }
+
+ public static void tearDown() {
+ try {
+ cc.close();
+ } catch (IOException e) {
+ LOG.warn("Unexpected, ignoring", e);
+
+ }
+ cc = null;
+ try {
+ cs.stop();
+ } catch (IOException e) {
+ LOG.warn("Unexpected, ignoring", e);
+
+ }
+ cs = null;
+ }
+
+ public static MBeanServerConnection conn() throws IOException {
+ return cc.getMBeanServerConnection();
+ }
+
+ /**
+ * Ensure that all of the specified names are registered.
+ * Note that these are components of the name, and in particular
+ * order matters - you want the more specific name (leafs) specified
+ * before their parent(s) (since names are hierarchical)
+ * @param expectedNames
+ * @return
+ * @throws IOException
+ * @throws MalformedObjectNameException
+ */
+ public static Set<ObjectName> ensureAll(String... expectedNames)
+ throws IOException
+ {
+ Set<ObjectName> beans;
+ try {
+ beans = conn().queryNames(
+ new ObjectName(CommonNames.DOMAIN + ":*"), null);
+ } catch (MalformedObjectNameException e) {
+ throw new RuntimeException(e);
+ }
+
+ Set<ObjectName> found = new HashSet<ObjectName>();
+ for (String name : expectedNames) {
+ System.err.println("expect:" + name);
+ for (ObjectName bean : beans) {
+ if (bean.toString().contains(name)) {
+ System.err.println("found:" + name + " " + bean);
+ found.add(bean);
+ break;
+ }
+ }
+ beans.removeAll(found);
+ }
+ TestCase.assertEquals("expected " + Arrays.toString(expectedNames),
+ expectedNames.length, found.size());
+ return beans;
+ }
+
+ /**
+ * Ensure that only the specified names are registered.
+ * Note that these are components of the name, and in particular
+ * order matters - you want the more specific name (leafs) specified
+ * before their parent(s) (since names are hierarchical)
+ * @param expectedNames
+ * @return
+ * @throws IOException
+ * @throws MalformedObjectNameException
+ */
+ public static Set<ObjectName> ensureOnly(String... expectedNames)
+ throws IOException
+ {
+ System.err.println("ensureOnly:" + Arrays.toString(expectedNames));
+ Set<ObjectName> beans = ensureAll(expectedNames);
+ for (ObjectName bean : beans) {
+ System.err.println("unexpected:" + bean.toString());
+ }
+ TestCase.assertEquals(0, beans.size());
+ return beans;
+ }
+
+ public static void ensureNone(String... expectedNames)
+ throws IOException
+ {
+ Set<ObjectName> beans;
+ try {
+ beans = conn().queryNames(
+ new ObjectName(CommonNames.DOMAIN + ":*"), null);
+ } catch (MalformedObjectNameException e) {
+ throw new RuntimeException(e);
+ }
+
+ for (String name : expectedNames) {
+ for (ObjectName bean : beans) {
+ if (bean.toString().contains(name)) {
+ System.err.println("didntexpect:" + name);
+ TestCase.fail(name + " " + bean.toString());
+ }
+ }
+ }
+ }
+
+ public static void dump() throws IOException {
+ System.err.println("JMXEnv.dump() follows");
+ Set<ObjectName> beans;
+ try {
+ beans = conn().queryNames(
+ new ObjectName(CommonNames.DOMAIN + ":*"), null);
+ } catch (MalformedObjectNameException e) {
+ throw new RuntimeException(e);
+ }
+ for (ObjectName bean : beans) {
+ System.err.println("bean:" + bean.toString());
+ }
+ }
+
+}
Modified: hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/QuorumBase.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/QuorumBase.java?rev=774909&r1=774908&r2=774909&view=diff
==============================================================================
--- hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/QuorumBase.java (original)
+++ hadoop/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/QuorumBase.java Thu May
14 20:15:06 2009
@@ -22,6 +22,8 @@
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.Set;
import org.apache.log4j.Logger;
import org.apache.zookeeper.ZooKeeper;
@@ -40,6 +42,8 @@
protected void setUp() throws Exception {
LOG.info("STARTING " + getName());
+ JMXEnv.setUp();
+
setupTestEnv();
hostPort = "127.0.0.1:2181,127.0.0.1:2182,127.0.0.1:2183,127.0.0.1:2184,127.0.0.1:2185";
@@ -54,7 +58,7 @@
LOG.info("Setup finished");
}
- void startServers() throws IOException, InterruptedException {
+ void startServers() throws Exception {
int tickTime = 2000;
int initLimit = 3;
int syncLimit = 3;
@@ -98,6 +102,28 @@
CONNECTION_TIMEOUT));
LOG.info(hp + " is accepting client connections");
}
+
+ // interesting to see what's there...
+ JMXEnv.dump();
+ // make sure we have these 5 servers listed
+ Set<String> ensureNames = new LinkedHashSet<String>();
+ for (int i = 1; i <= 5; i++) {
+ ensureNames.add("InMemoryDataTree");
+ }
+ for (int i = 1; i <= 5; i++) {
+ ensureNames.add("name0=ReplicatedServer_id" + i
+ + ",name1=replica." + i + ",name2=");
+ }
+ for (int i = 1; i <= 5; i++) {
+ for (int j = 1; j <= 5; j++) {
+ ensureNames.add("name0=ReplicatedServer_id" + i
+ + ",name1=replica." + j);
+ }
+ }
+ for (int i = 1; i <= 5; i++) {
+ ensureNames.add("name0=ReplicatedServer_id" + i);
+ }
+ JMXEnv.ensureAll(ensureNames.toArray(new String[ensureNames.size()]));
}
@After
@@ -117,6 +143,8 @@
LOG.info(hp + " is no longer accepting client connections");
}
+ JMXEnv.tearDown();
+
LOG.info("FINISHED " + getName());
}
|