Browse Source

ZOOKEEPER-1728. Better error message when reconfig invoked in standalone mode (Alexander Shraer via michim)

git-svn-id: https://svn.apache.org/repos/asf/zookeeper/trunk@1583492 13f79535-47bb-0310-9956-ffa450edef68
Michi Mutsuzaki 11 years ago
parent
commit
704af2e35c

+ 3 - 0
CHANGES.txt

@@ -823,6 +823,9 @@ IMPROVEMENTS:
   READY_START event even if the ELECTED node stopped/expired simultaneously.
   (Rakesh R via michim)
 
+  ZOOKEEPER-1728. Better error message when reconfig invoked in standalone mode
+  (Alexander Shraer via michim)
+
 headers
 
 Release 3.4.0 - 

+ 8 - 2
src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java

@@ -513,8 +513,14 @@ public class PrepRequestProcessor extends ZooKeeperCriticalThread implements
                 break;
             case OpCode.reconfig:
                 zks.sessionTracker.checkSession(request.sessionId, request.getOwner());
-                ReconfigRequest reconfigRequest = (ReconfigRequest)record;                             
-                LeaderZooKeeperServer lzks = (LeaderZooKeeperServer)zks;
+                ReconfigRequest reconfigRequest = (ReconfigRequest)record; 
+                LeaderZooKeeperServer lzks;
+                try {
+                    lzks = (LeaderZooKeeperServer)zks;
+                } catch (ClassCastException e) {
+                    // standalone mode - reconfiguration currently not supported
+                    throw new KeeperException.UnimplementedException();
+                }
                 QuorumVerifier lastSeenQV = lzks.self.getLastSeenQuorumVerifier();                                                                                 
                 // check that there's no reconfig in progress
                 if (lastSeenQV.getVersion()!=lzks.self.getQuorumVerifier().getVersion()) {

+ 50 - 1
src/java/test/org/apache/zookeeper/test/StandaloneTest.java

@@ -20,14 +20,23 @@ package org.apache.zookeeper.test;
 
 import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
 
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.server.ZooKeeperServer;
 import org.apache.zookeeper.server.quorum.QuorumPeerTestBase;
+import org.apache.zookeeper.test.ClientBase.CountdownWatcher;
 import org.junit.Assert;
 import org.junit.Test;
-import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
 
 /**
  * Standalone server tests.
@@ -65,4 +74,44 @@ public class StandaloneTest extends QuorumPeerTestBase implements Watcher{
         }
     }    
     
+    /**
+     * Verify that reconfiguration in standalone mode fails with
+     * KeeperException.UnimplementedException.
+     */
+    @Test
+    public void testStandaloneReconfigFails() throws Exception {
+        ClientBase.setupTestEnv();
+
+        final int CLIENT_PORT = PortAssignment.unique();
+        final String HOSTPORT = "127.0.0.1:" + CLIENT_PORT;
+
+        File tmpDir = ClientBase.createTmpDir();
+        ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+
+        ServerCnxnFactory f = ServerCnxnFactory.createFactory(CLIENT_PORT, -1);
+        f.startup(zks);
+        Assert.assertTrue("waiting for server being up ", ClientBase
+                .waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));
+
+        CountdownWatcher watcher = new CountdownWatcher();
+        ZooKeeper zk = new ZooKeeper(HOSTPORT, CONNECTION_TIMEOUT, watcher);
+        watcher.waitForConnected(CONNECTION_TIMEOUT);
+
+        List<String> joiners = new ArrayList<String>();
+        joiners.add("server.2=localhost:1234:1235;1236");
+        // generate some transactions that will get logged
+        try {
+            zk.reconfig(joiners, null, null, -1, new Stat());
+            Assert.fail("Reconfiguration in standalone should trigger " +
+                        "UnimplementedException");
+        } catch (KeeperException.UnimplementedException ex) {
+            // expected
+        }
+        zk.close();
+
+        zks.shutdown();
+        f.shutdown();
+        Assert.assertTrue("waiting for server being down ", ClientBase
+                .waitForServerDown(HOSTPORT, CONNECTION_TIMEOUT));
+    }
 }