Author: phunt
Date: Tue Sep 14 21:07:22 2010
New Revision: 997097
URL: http://svn.apache.org/viewvc?rev=997097&view=rev
Log:
Zookeeper 3.3.1 shouldn't infinite loop if someone creates a server.0 line (part 2)
Added:
hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/test/StandaloneTest.java
Modified:
hadoop/zookeeper/branches/branch-3.3/CHANGES.txt
hadoop/zookeeper/branches/branch-3.3/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java
hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/test/ObserverTest.java
hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/test/QuorumTest.java
Modified: hadoop/zookeeper/branches/branch-3.3/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/branches/branch-3.3/CHANGES.txt?rev=997097&r1=997096&r2=997097&view=diff
==============================================================================
--- hadoop/zookeeper/branches/branch-3.3/CHANGES.txt (original)
+++ hadoop/zookeeper/branches/branch-3.3/CHANGES.txt Tue Sep 14 21:07:22 2010
@@ -12,11 +12,16 @@ BUGFIXES:
ZOOKEEPER-772. zkpython segfaults when watcher from async get children
is invoked. (henry robinson via mahadev)
- ZOOKEEPER-795. eventThread isn't shutdown after a connection "session expired" event coming
(sergey doroshenko via breed)
+ ZOOKEEPER-795. eventThread isn't shutdown after a connection
+ "session expired" event coming (sergey doroshenko via breed)
ZOOKEEPER-792. zkpython memory leak (Lei Zhang via henryr)
- ZOOKEEPER-785. Zookeeper 3.3.1 shouldn't infinite loop if someone creates a server.0 line
(phunt and Andrei Savu via breed)
+ ZOOKEEPER-785. Zookeeper 3.3.1 shouldn't infinite loop if someone creates a
+ server.0 line (phunt and Andrei Savu via breed)
+
+ ZOOKEEPER-785. Zookeeper 3.3.1 shouldn't infinite loop if someone creates a
+ server.0 line (part 2) (phunt)
IMPROVEMENTS:
Modified: hadoop/zookeeper/branches/branch-3.3/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/branches/branch-3.3/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java?rev=997097&r1=997096&r2=997097&view=diff
==============================================================================
--- hadoop/zookeeper/branches/branch-3.3/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
(original)
+++ hadoop/zookeeper/branches/branch-3.3/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
Tue Sep 14 21:07:22 2010
@@ -241,13 +241,29 @@ public class QuorumPeerConfig {
throw new IllegalArgumentException(
"minSessionTimeout must not be larger than maxSessionTimeout");
}
- if (servers.size() == 1) {
+ if (servers.size() == 0) {
+ if (observers.size() > 0) {
+ throw new IllegalArgumentException("Observers w/o participants is an invalid
configuration");
+ }
+ // Not a quorum configuration so return immediately - not an error
+ // case (for b/w compatibility), server will default to standalone
+ // mode.
+ return;
+ } else if (servers.size() == 1) {
+ if (observers.size() > 0) {
+ throw new IllegalArgumentException("Observers w/o quorum is an invalid configuration");
+ }
+
+ // HBase currently adds a single server line to the config, for
+ // b/w compatibility reasons we need to keep this here.
LOG.error("Invalid configuration, only one server specified (ignoring)");
servers.clear();
} else if (servers.size() > 1) {
if (servers.size() == 2) {
LOG.warn("No server failure will be tolerated. " +
"You need at least 3 servers.");
+ } else if (servers.size() % 2 == 0) {
+ LOG.warn("Non-optimial configuration, consider an odd number of servers.");
}
if (initLimit == 0) {
throw new IllegalArgumentException("initLimit is not set");
@@ -298,7 +314,7 @@ public class QuorumPeerConfig {
// Now add observers to servers, once the quorums have been
// figured out
servers.putAll(observers);
-
+
File myIdFile = new File(dataDir, "myid");
if (!myIdFile.exists()) {
throw new IllegalArgumentException(myIdFile.toString()
Modified: hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java?rev=997097&r1=997096&r2=997097&view=diff
==============================================================================
--- hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java
(original)
+++ hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java
Tue Sep 14 21:07:22 2010
@@ -48,7 +48,9 @@ public class QuorumPeerTestBase extends
public static class TestQPMain extends QuorumPeerMain {
public void shutdown() {
// ensure it closes - in particular wait for thread to exit
- QuorumBase.shutdown(quorumPeer);
+ if (quorumPeer != null) {
+ QuorumBase.shutdown(quorumPeer);
+ }
}
}
Modified: hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/test/ObserverTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/test/ObserverTest.java?rev=997097&r1=997096&r2=997097&view=diff
==============================================================================
--- hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/test/ObserverTest.java
(original)
+++ hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/test/ObserverTest.java
Tue Sep 14 21:07:22 2010
@@ -20,20 +20,20 @@ package org.apache.zookeeper.test;
import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
-import java.io.IOException;
import java.util.concurrent.CountDownLatch;
import org.apache.log4j.Logger;
import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException.ConnectionLossException;
import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.KeeperException.ConnectionLossException;
import org.apache.zookeeper.Watcher.Event.KeeperState;
import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.ZooKeeper.States;
import org.apache.zookeeper.server.quorum.QuorumPeerTestBase;
+import org.junit.Assert;
import org.junit.Test;
/**
@@ -181,25 +181,42 @@ public class ObserverTest extends Quorum
/**
* This test ensures that an Observer does not elect itself as a leader, or
* indeed come up properly, if it is the lone member of an ensemble.
- * @throws IOException
+ * @throws Exception
*/
@Test
- public void testSingleObserver() throws IOException{
+ public void testObserverOnly() throws Exception {
ClientBase.setupTestEnv();
final int CLIENT_PORT_QP1 = PortAssignment.unique();
- final int CLIENT_PORT_QP2 = PortAssignment.unique();
String quorumCfgSection =
- "server.1=127.0.0.1:" + (CLIENT_PORT_QP1)
- + ":" + (CLIENT_PORT_QP2) + "\npeerType=observer";
+ "server.1=127.0.0.1:" + (PortAssignment.unique())
+ + ":" + (PortAssignment.unique()) + ":observer\npeerType=observer\n";
MainThread q1 = new MainThread(1, CLIENT_PORT_QP1, quorumCfgSection);
q1.start();
- assertFalse("Observer shouldn't come up",
- ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1,
- CONNECTION_TIMEOUT));
-
- q1.shutdown();
+ q1.join(ClientBase.CONNECTION_TIMEOUT);
+ Assert.assertFalse(q1.isAlive());
}
+ /**
+ * Ensure that observer only comes up when a proper ensemble is configured.
+ * (and will not come up with standalone server).
+ */
+ @Test
+ public void testObserverWithStandlone() throws Exception {
+ ClientBase.setupTestEnv();
+ final int CLIENT_PORT_QP1 = PortAssignment.unique();
+
+ String quorumCfgSection =
+ "server.1=127.0.0.1:" + (PortAssignment.unique())
+ + ":" + (PortAssignment.unique()) + ":observer\n"
+ + "server.2=127.0.0.1:" + (PortAssignment.unique())
+ + ":" + (PortAssignment.unique()) + "\npeerType=observer\n";
+
+ MainThread q1 = new MainThread(1, CLIENT_PORT_QP1, quorumCfgSection);
+ q1.start();
+ q1.join(ClientBase.CONNECTION_TIMEOUT);
+ Assert.assertFalse(q1.isAlive());
+ }
+
}
Modified: hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/test/QuorumTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/test/QuorumTest.java?rev=997097&r1=997096&r2=997097&view=diff
==============================================================================
--- hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/test/QuorumTest.java
(original)
+++ hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/test/QuorumTest.java
Tue Sep 14 21:07:22 2010
@@ -19,9 +19,8 @@
package org.apache.zookeeper.test;
import java.io.IOException;
import java.util.ArrayList;
-import java.util.concurrent.TimeUnit;
import java.util.concurrent.Semaphore;
-
+import java.util.concurrent.TimeUnit;
import org.apache.log4j.Logger;
import org.apache.zookeeper.AsyncCallback;
@@ -385,6 +384,5 @@ public class QuorumTest extends QuorumBa
}
-
// skip superhammer and clientcleanup as they are too expensive for quorum
}
Added: hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/test/StandaloneTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/test/StandaloneTest.java?rev=997097&view=auto
==============================================================================
--- hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/test/StandaloneTest.java
(added)
+++ hadoop/zookeeper/branches/branch-3.3/src/java/test/org/apache/zookeeper/test/StandaloneTest.java
Tue Sep 14 21:07:22 2010
@@ -0,0 +1,64 @@
+/**
+ * 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 org.apache.log4j.Logger;
+import org.apache.zookeeper.PortAssignment;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.server.quorum.QuorumPeerTestBase;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Standalone server tests.
+ */
+public class StandaloneTest extends QuorumPeerTestBase implements Watcher{
+ protected static final Logger LOG =
+ Logger.getLogger(StandaloneTest.class);
+
+ /**
+ * Ensure that a single standalone server comes up when misconfigured
+ * with a single server.# line in the configuration. This handles the
+ * case of HBase, which configures zoo.cfg in this way. Maintain b/w
+ * compatibility.
+ * TODO remove in a future version (4.0.0 hopefully)
+ */
+ @Test
+ public void testStandaloneQuorum() throws Exception {
+ ClientBase.setupTestEnv();
+ final int CLIENT_PORT_QP1 = PortAssignment.unique();
+
+ String quorumCfgSection =
+ "server.1=127.0.0.1:" + (PortAssignment.unique())
+ + ":" + (PortAssignment.unique()) + "\n";
+
+ MainThread q1 = new MainThread(1, CLIENT_PORT_QP1, quorumCfgSection);
+ q1.start();
+ try {
+ Assert.assertTrue("waiting for server 1 being up",
+ ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1,
+ CONNECTION_TIMEOUT));
+ } finally {
+ q1.shutdown();
+ }
+ }
+
+}
|