You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ra...@apache.org on 2014/07/26 04:11:38 UTC

svn commit: r1613553 - in /zookeeper/trunk: ./ src/java/main/org/apache/zookeeper/common/ src/java/main/org/apache/zookeeper/server/quorum/ src/java/test/org/apache/zookeeper/server/ src/java/test/org/apache/zookeeper/server/quorum/ src/java/test/org/a...

Author: rakeshr
Date: Sat Jul 26 02:11:37 2014
New Revision: 1613553

URL: http://svn.apache.org/r1613553
Log:
ZOOKEEPER-1988. new test patch to verify dynamic reconfig backward compatibility (Alexander Shraer via rakeshr)

Added:
    zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/DynamicConfigBackwardCompatibilityTest.java
Removed:
    zookeeper/trunk/src/java/test/org/apache/zookeeper/server/util/DynamicConfigBCTest.java
Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/src/java/main/org/apache/zookeeper/common/PathUtils.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/server/InvalidSnapCountTest.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1613553&r1=1613552&r2=1613553&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Sat Jul 26 02:11:37 2014
@@ -747,6 +747,9 @@ BUGFIXES:
   ZOOKEEPER-1933. Windows release build of zk client cannot connect to
   zk server (Orion Hodson via fpj, phunt)
 
+  ZOOKEEPER-1988. new test patch to verify dynamic reconfig backward
+  compatibility (Alexander Shraer via rakeshr)
+
 IMPROVEMENTS:
 
   ZOOKEEPER-1170. Fix compiler (eclipse) warnings: unused imports,

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/common/PathUtils.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/common/PathUtils.java?rev=1613553&r1=1613552&r2=1613553&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/common/PathUtils.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/common/PathUtils.java Sat Jul 26 02:11:37 2014
@@ -100,4 +100,21 @@ public class PathUtils {
                     "Invalid path string \"" + path + "\" caused by " + reason);
         }
     }
+
+    /**
+     * Convert Windows path to Unix
+     *
+     * @param path
+     *            file path
+     * @return converted file path
+     */
+    public static String normalizeFileSystemPath(String path) {
+        if (path != null) {
+            String osname = java.lang.System.getProperty("os.name");
+            if (osname.toLowerCase().contains("windows")) {
+                return path.replace('\\', '/');
+            }
+        }
+        return path;
+    }
 }

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java?rev=1613553&r1=1613552&r2=1613553&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeer.java Sat Jul 26 02:11:37 2014
@@ -45,12 +45,12 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
-
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.common.AtomicFileWritingIdiom;
 import org.apache.zookeeper.common.AtomicFileWritingIdiom.WriterStatement;
 import org.apache.zookeeper.common.HostNameUtils;
+import org.apache.zookeeper.common.PathUtils;
 import org.apache.zookeeper.jmx.MBeanRegistry;
 import org.apache.zookeeper.jmx.ZKMBeanInfo;
 import org.apache.zookeeper.server.DataNode;
@@ -319,7 +319,7 @@ public class QuorumPeer extends ZooKeepe
 
        
     protected synchronized void setDynamicConfigFilename(String s) {
-        dynamicConfigFilename = s;
+        dynamicConfigFilename = PathUtils.normalizeFileSystemPath(s);
     }
 
     protected synchronized String getDynamicConfigFilename() {
@@ -1341,11 +1341,11 @@ public class QuorumPeer extends ZooKeepe
                 // so we'll create the dynamic config file for the first time now
                 if (dynamicConfigFilename !=null || (configFilename !=null && configBackwardCompatibility)) { 
                 try {
-                    QuorumPeerConfig.writeDynamicConfig(dynamicConfigFilename, configFilename, configBackwardCompatibility, qv);
                     if (configBackwardCompatibility) {
-                        dynamicConfigFilename = configFilename + ".dynamic";
-                        configBackwardCompatibility = false;
+                        setDynamicConfigFilename(configFilename + ".dynamic");
                     }
+                    QuorumPeerConfig.writeDynamicConfig(dynamicConfigFilename, configFilename, configBackwardCompatibility, qv);
+                    configBackwardCompatibility = false;
                 } catch(IOException e){
                     LOG.error("Error closing file: ", e.getMessage());     
                 }

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java?rev=1613553&r1=1613552&r2=1613553&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java Sat Jul 26 02:11:37 2014
@@ -304,9 +304,6 @@ public class QuorumPeerConfig {
     public static void writeDynamicConfig(String dynamicConfigFilename, String configFileStr, 
             boolean configBackwardCompatibilityMode, final QuorumVerifier qv) throws IOException {
 
-        if (configBackwardCompatibilityMode) {
-            dynamicConfigFilename = configFileStr + ".dynamic";
-        }
         final String actualDynamicConfigFilename = dynamicConfigFilename;
         new AtomicFileWritingIdiom(new File(actualDynamicConfigFilename), new OutputStreamStatement() {
             @Override

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/server/InvalidSnapCountTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/InvalidSnapCountTest.java?rev=1613553&r1=1613552&r2=1613553&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/server/InvalidSnapCountTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/server/InvalidSnapCountTest.java Sat Jul 26 02:11:37 2014
@@ -30,6 +30,7 @@ import org.apache.zookeeper.WatchedEvent
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.common.PathUtils;
 import org.apache.zookeeper.test.ClientBase;
 import org.junit.Assert;
 import org.junit.Test;
@@ -63,11 +64,7 @@ public class InvalidSnapCountTest extend
             }
             
             // Convert windows path to UNIX to avoid problems with "\"
-            String dir = dataDir.toString();
-            String osname = java.lang.System.getProperty("os.name");
-            if (osname.toLowerCase().contains("windows")) {
-                dir = dir.replace('\\', '/');
-            }
+            String dir = PathUtils.normalizeFileSystemPath(dataDir.toString());
             fwriter.write("dataDir=" + dir + "\n");
             
             fwriter.write("clientPort=" + clientPort + "\n");

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java?rev=1613553&r1=1613552&r2=1613553&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/server/ZooKeeperServerMainTest.java Sat Jul 26 02:11:37 2014
@@ -37,6 +37,7 @@ import org.apache.zookeeper.ZKTestCase;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.common.PathUtils;
 import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
 import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
 import org.apache.zookeeper.test.ClientBase;
@@ -82,12 +83,8 @@ public class ZooKeeperServerMainTest ext
                 dirLog = dataDir.toString();
             }
             
-            // Convert windows path to UNIX to avoid problems with "\"
-            String osname = java.lang.System.getProperty("os.name");
-            if (osname.toLowerCase().contains("windows")) {
-                dir = dir.replace('\\', '/');
-                dirLog = dirLog.replace('\\', '/');
-            }
+            dir = PathUtils.normalizeFileSystemPath(dir);
+            dirLog = PathUtils.normalizeFileSystemPath(dirLog);
             fwriter.write("dataDir=" + dir + "\n");
             fwriter.write("dataLogDir=" + dirLog + "\n");
             fwriter.write("clientPort=" + clientPort + "\n");

Added: zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/DynamicConfigBackwardCompatibilityTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/DynamicConfigBackwardCompatibilityTest.java?rev=1613553&view=auto
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/DynamicConfigBackwardCompatibilityTest.java (added)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/DynamicConfigBackwardCompatibilityTest.java Sat Jul 26 02:11:37 2014
@@ -0,0 +1,140 @@
+/**
+ * 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.server.quorum;
+
+import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Properties;
+
+import org.apache.zookeeper.PortAssignment;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.test.ClientBase;
+import org.apache.zookeeper.test.ReconfigTest;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class DynamicConfigBackwardCompatibilityTest extends QuorumPeerTestBase {
+    /**
+     * This test checks that when started with a single static config file the
+     * servers will create a valid dynamic config file.
+     */
+    @Test
+    public void testConfigFileBackwardCompatibility() throws Exception {
+        ClientBase.setupTestEnv();
+
+        final int SERVER_COUNT = 3;
+        final int clientPorts[] = new int[SERVER_COUNT];
+        StringBuilder sb = new StringBuilder();
+        String server;
+        ArrayList<String> allServers = new ArrayList<String>();
+
+        for (int i = 0; i < SERVER_COUNT; i++) {
+            clientPorts[i] = PortAssignment.unique();
+            server = "server." + i + "=localhost:" + PortAssignment.unique()
+                    + ":" + PortAssignment.unique() + ":participant;localhost:"
+                    + clientPorts[i];
+            allServers.add(server);
+            sb.append(server + "\n");
+        }
+        String currentQuorumCfgSection = sb.toString();
+
+        MainThread mt[] = new MainThread[SERVER_COUNT];
+        ZooKeeper zk[] = new ZooKeeper[SERVER_COUNT];
+        
+        // Start the servers with the static config file, without the dynamic
+        // config file.
+        for (int i = 0; i < SERVER_COUNT; i++) {
+            mt[i] = new MainThread(i, clientPorts[i], currentQuorumCfgSection,
+                    false);
+            // check that a dynamic configuration file wasn't created
+            Assert.assertFalse(mt[i].dynamicConfigFile.exists());
+            mt[i].start();
+        }
+        // Check that the servers are up, have the right config and can process
+        // operations
+        // Check that the static config was split to static and dynamic files
+        // correctly
+        for (int i = 0; i < SERVER_COUNT; i++) {
+            Assert.assertTrue("waiting for server " + i + " being up",
+                    ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i],
+                            CONNECTION_TIMEOUT));
+            zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i],
+                    ClientBase.CONNECTION_TIMEOUT, this);
+            Assert.assertTrue(mt[i].dynamicConfigFile.exists());
+            ReconfigTest.testServerHasConfig(zk[i], allServers, null);
+            // check that static config file doesn't include membership info
+            // and has a pointer to the dynamic configuration file
+            Properties cfg = readPropertiesFromFile(mt[i].confFile);
+            for (int j = 0; j < SERVER_COUNT; j++) {
+                Assert.assertFalse(cfg.containsKey("server." + j));
+            }
+            Assert.assertTrue(cfg.containsKey("dynamicConfigFile"));
+            Assert.assertFalse(cfg.containsKey("clientPort"));
+
+            // check that the dynamic configuration file contains the membership
+            // info
+            cfg = readPropertiesFromFile(mt[i].dynamicConfigFile);
+            for (int j = 0; j < SERVER_COUNT; j++) {
+                String serverLine = cfg.getProperty("server." + j, "");
+                Assert.assertEquals(allServers.get(j), "server." + j + "="
+                        + serverLine);
+            }
+            Assert.assertFalse(cfg.containsKey("dynamicConfigFile"));
+        }
+        ReconfigTest.testNormalOperation(zk[0], zk[1]);
+
+        // now shut down the servers and restart them
+        for (int i = 0; i < SERVER_COUNT; i++) {
+            zk[i].close();
+            mt[i].shutdown();
+        }
+        for (int i = 0; i < SERVER_COUNT; i++) {
+            mt[i].start();
+        }
+        for (int i = 0; i < SERVER_COUNT; i++) {
+            Assert.assertTrue("waiting for server " + i + " being up",
+                    ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i],
+                            CONNECTION_TIMEOUT));
+            zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i],
+                    ClientBase.CONNECTION_TIMEOUT, this);
+            ReconfigTest.testServerHasConfig(zk[i], allServers, null);
+        }
+        ReconfigTest.testNormalOperation(zk[0], zk[1]);
+        for (int i = 0; i < SERVER_COUNT; i++) {
+            mt[i].shutdown();
+            zk[i].close();
+        }
+    }
+
+    private Properties readPropertiesFromFile(File file) throws IOException {
+        Properties cfg = new Properties();
+        FileInputStream in = new FileInputStream(file);
+        try {
+            cfg.load(in);
+        } finally {
+            in.close();
+        }
+        return cfg;
+    }
+}

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java?rev=1613553&r1=1613552&r2=1613553&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java Sat Jul 26 02:11:37 2014
@@ -30,6 +30,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZKTestCase;
+import org.apache.zookeeper.common.PathUtils;
 import org.apache.zookeeper.server.admin.JettyAdminServer;
 import org.apache.zookeeper.test.ClientBase;
 import org.apache.zookeeper.test.QuorumBase;
@@ -64,16 +65,26 @@ public class QuorumPeerTestBase extends 
 
         public MainThread(int myid, int clientPort, String quorumCfgSection)
                 throws IOException {
-            this(myid, clientPort, JettyAdminServer.DEFAULT_PORT, quorumCfgSection, null);
+            this(myid, clientPort, JettyAdminServer.DEFAULT_PORT, quorumCfgSection, null, true);
+        }
+
+        public MainThread(int myid, int clientPort, String quorumCfgSection, boolean writeDynamicConfigFile)
+                throws IOException {
+            this(myid, clientPort, JettyAdminServer.DEFAULT_PORT, quorumCfgSection, null, writeDynamicConfigFile);
         }
 
         public MainThread(int myid, int clientPort, String quorumCfgSection, String configs)
                 throws IOException {
-            this(myid, clientPort, JettyAdminServer.DEFAULT_PORT, quorumCfgSection, configs);
+            this(myid, clientPort, JettyAdminServer.DEFAULT_PORT, quorumCfgSection, configs, true);
         }
 
         public MainThread(int myid, int clientPort, int adminServerPort, String quorumCfgSection,
-                String configs)
+                String configs)  throws IOException {
+            this(myid, clientPort, adminServerPort, quorumCfgSection, configs, true);
+        }
+
+        public MainThread(int myid, int clientPort, int adminServerPort, String quorumCfgSection,
+                String configs, boolean writeDynamicConfigFile)
                 throws IOException {
             tmpDir = ClientBase.createTmpDir();
             LOG.info("id = " + myid + " tmpDir = " + tmpDir + " clientPort = "
@@ -85,7 +96,7 @@ public class QuorumPeerTestBase extends 
             }
 
             confFile = new File(tmpDir, "zoo.cfg");
-            dynamicConfigFile = new File(tmpDir, "zoo.dynamic");
+            dynamicConfigFile = new File(tmpDir, "zoo.cfg.dynamic");
 
             FileWriter fwriter = new FileWriter(confFile);
             fwriter.write("tickTime=4000\n");
@@ -96,27 +107,24 @@ public class QuorumPeerTestBase extends 
             }
 
             // Convert windows path to UNIX to avoid problems with "\"
-            String dir = dataDir.toString();
-            String dynamicConfigFilename = dynamicConfigFile.toString();
-            String osname = java.lang.System.getProperty("os.name");
-            if (osname.toLowerCase().contains("windows")) {
-                dir = dir.replace('\\', '/');
-                dynamicConfigFilename = dynamicConfigFilename
-                        .replace('\\', '/');
-            }
+            String dir = PathUtils.normalizeFileSystemPath(dataDir.toString());
+
             fwriter.write("dataDir=" + dir + "\n");
 
             fwriter.write("clientPort=" + clientPort + "\n");
 
             fwriter.write("admin.serverPort=" + adminServerPort + "\n");
 
-            fwriter.write("dynamicConfigFile=" + dynamicConfigFilename + "\n");
-
-            fwriter.flush();
-            fwriter.close();
-
-            fwriter = new FileWriter(dynamicConfigFile);
-            fwriter.write(quorumCfgSection + "\n");
+            if (writeDynamicConfigFile) {
+                String dynamicConfigFilename = PathUtils.normalizeFileSystemPath(dynamicConfigFile.toString());
+                fwriter.write("dynamicConfigFile=" + dynamicConfigFilename + "\n");
+                FileWriter fDynamicConfigWriter = new FileWriter(dynamicConfigFile);
+                fDynamicConfigWriter.write(quorumCfgSection + "\n");
+                fDynamicConfigWriter.flush();
+                fDynamicConfigWriter.close();
+            } else {
+                fwriter.write(quorumCfgSection + "\n");
+            }
             fwriter.flush();
             fwriter.close();
 
@@ -129,7 +137,7 @@ public class QuorumPeerTestBase extends 
 
         public void writeTempDynamicConfigFile(String nextQuorumCfgSection)
                 throws IOException {
-            File nextDynamicConfigFile = new File(tmpDir, "zoo.dynamic.next");
+            File nextDynamicConfigFile = new File(tmpDir, "zoo.cfg.dynamic.next");
             FileWriter fwriter = new FileWriter(nextDynamicConfigFile);
             fwriter.write(nextQuorumCfgSection + "\n");
             fwriter.flush();