You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by sh...@apache.org on 2014/08/19 22:50:42 UTC

svn commit: r1618977 - in /zookeeper/trunk: ./ bin/ src/java/main/org/apache/zookeeper/server/quorum/ src/java/test/org/apache/zookeeper/server/quorum/ src/java/test/org/apache/zookeeper/test/

Author: shralex
Date: Tue Aug 19 20:50:41 2014
New Revision: 1618977

URL: http://svn.apache.org/r1618977
Log:
ZOOKEEPER-2006. Standalone mode won't take client port from dynamic config. (Hongchao Deng via Alex Shraer)

Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/bin/zkServer.sh
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/QuorumPeerConfig.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerTestBase.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/StandaloneTest.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1618977&r1=1618976&r2=1618977&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Tue Aug 19 20:50:41 2014
@@ -43,6 +43,8 @@ NEW FEATURES:
   (Skye Wanderman-Milne, Bill Havanki via phunt)
 
 BUGFIXES:
+  ZOOKEEPER-2006. Standalone mode won't take client port from dynamic config. (Hongchao Deng via Alex Shraer)  
+
   ZOOKEEPER-2008. Missing leader election port in system test. (Kfir Lev-Ari via Alex Shraer).
 
   ZOOKEEPER-1992. Backward compatibility of the static configuration file (Hongchao Deng via Alex Shraer).

Modified: zookeeper/trunk/bin/zkServer.sh
URL: http://svn.apache.org/viewvc/zookeeper/trunk/bin/zkServer.sh?rev=1618977&r1=1618976&r2=1618977&view=diff
==============================================================================
--- zookeeper/trunk/bin/zkServer.sh (original)
+++ zookeeper/trunk/bin/zkServer.sh Tue Aug 19 20:50:41 2014
@@ -179,19 +179,34 @@ status)
 	clientPortAddress="localhost"
     fi
     clientPort=`grep "^[[:space:]]*clientPort[^[:alpha:]]" "$ZOOCFG" | sed -e 's/.*=//'`
-    if ! [ $clientPort ]
+    if ! [[ "$clientPort"  =~ ^[0-9]+$ ]]
     then
-       echo "Client port not found in static config file. Looking in dynamic config file."
        dataDir=`grep "^[[:space:]]*dataDir" "$ZOOCFG" | sed -e 's/.*=//'`
        myid=`cat "$dataDir/myid"`
-       dynamicConfigFile=`grep "^[[:space:]]*dynamicConfigFile" "$ZOOCFG" | sed -e 's/.*=//'`
-       clientPort=`grep "^[[:space:]]*server.$myid" "$dynamicConfigFile" | sed -e 's/.*=//' | sed -e 's/.*;//' | sed -e 's/.*://'`
-       if ! [[ "$clientPort" =~ ^[0-9]+$ ]] ; then
+       if ! [[ "$myid" =~ ^[0-9]+$ ]] ; then
+         echo "clientPort not found and myid could not be determined. Terminating."
+         exit 1
+       fi
+       clientPortAndAddress=`grep "^[[:space:]]*server.$myid=.*;.*" "$ZOOCFG" | sed -e 's/.*=//' | sed -e 's/.*;//'`
+       if [ ! "$clientPortAndAddress" ] ; then
+           echo "Client port not found in static config file. Looking in dynamic config file."
+           dynamicConfigFile=`grep "^[[:space:]]*dynamicConfigFile" "$ZOOCFG" | sed -e 's/.*=//'`
+           clientPortAndAddress=`grep "^[[:space:]]*server.$myid=.*;.*" "$dynamicConfigFile" | sed -e 's/.*=//' | sed -e 's/.*;//'`
+       fi
+       if [ ! "$clientPortAndAddress" ] ; then
+          echo "Client port not found. Terminating."
+          exit 1
+       fi
+       if [[ "$clientPortAndAddress" =~ ^.*:[0-9]+ ]] ; then
+          clientPortAddress=`echo "$clientPortAndAddress" | sed -e 's/:.*//'`
+       fi
+       clientPort=`echo "$clientPortAndAddress" | sed -e 's/.*://'`
+       if [ ! "$clientPort" ] ; then
           echo "Client port not found. Terminating."
           exit 1
        fi
     fi
-    echo "Client port found: $clientPort"
+    echo "Client port found: $clientPort. Client address: $clientPortAddress."
     STAT=`"$JAVA" "-Dzookeeper.log.dir=${ZOO_LOG_DIR}" "-Dzookeeper.root.logger=${ZOO_LOG4J_PROP}" \
              -cp "$CLASSPATH" $JVMFLAGS org.apache.zookeeper.client.FourLetterWordMain \
              $clientPortAddress $clientPort srvr 2> /dev/null    \

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=1618977&r1=1618976&r2=1618977&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 Tue Aug 19 20:50:41 2014
@@ -54,10 +54,11 @@ import org.apache.zookeeper.server.util.
 
 public class QuorumPeerConfig {
     private static final Logger LOG = LoggerFactory.getLogger(QuorumPeerConfig.class);
-    private static boolean standaloneEnabled = true;
-
+    private static final int UNSET_SERVERID = -1;
     public static final String nextDynamicConfigFileSuffix = ".dynamic.next";
 
+    private static boolean standaloneEnabled = true;
+
     protected InetSocketAddress clientPortAddress;
     protected File dataDir;
     protected File dataLogDir;
@@ -78,7 +79,7 @@ public class QuorumPeerConfig {
     protected int electionPort = 2182;
     protected boolean quorumListenOnAllIPs = false;
 
-    protected long serverId;
+    protected long serverId = UNSET_SERVERID;
 
     protected QuorumVerifier quorumVerifier = null, lastSeenQuorumVerifier = null;
     protected int snapRetainCount = 3;
@@ -152,9 +153,8 @@ public class QuorumPeerConfig {
                } finally {
                    inConfig.close();
                }
-               quorumVerifier = parseDynamicConfig(dynamicCfg, electionAlg, true, false);
-               checkValidity();
-           
+               setupQuorumPeerConfig(dynamicCfg, false);
+
            } catch (IOException e) {
                throw new ConfigException("Error processing " + dynamicConfigFileStr, e);
            } catch (IllegalArgumentException e) {
@@ -318,9 +318,11 @@ public class QuorumPeerConfig {
         // backward compatibility - dynamic configuration in the same file as
         // static configuration params see writeDynamicConfig()
         if (dynamicConfigFileStr == null) {
-            backupOldConfig();
-            quorumVerifier = parseDynamicConfig(zkProp, electionAlg, true, true);
-            checkValidity();
+            setupQuorumPeerConfig(zkProp, true);
+            if (isDistributed()) {
+                // we don't backup static config for standalone mode.
+                backupOldConfig();
+            }
         }
     }
 
@@ -464,7 +466,16 @@ public class QuorumPeerConfig {
             return new QuorumMaj(dynamicConfigProp);            
         }          
     }
-    
+
+    void setupQuorumPeerConfig(Properties prop, boolean configBackwardCompatibilityMode)
+            throws IOException, ConfigException {
+        quorumVerifier = parseDynamicConfig(prop, electionAlg, true, configBackwardCompatibilityMode);
+        setupMyId();
+        setupClientPort();
+        setupPeerType();
+        checkValidity();
+    }
+
     /**
      * Parse dynamic configuration file and return
      * quorumVerifier for new configuration.
@@ -490,6 +501,10 @@ public class QuorumPeerConfig {
         int numParticipators = qv.getVotingMembers().size();
         int numObservers = qv.getObservingMembers().size();
         if (numParticipators == 0) {
+            if (!standaloneEnabled) {
+                throw new IllegalArgumentException("standaloneEnabled = false then " +
+                        "number of participants should be >0");
+            }
             if (numObservers > 0) {
                 throw new IllegalArgumentException("Observers w/o participants is an invalid configuration");
             }
@@ -525,64 +540,73 @@ public class QuorumPeerConfig {
         }
         return qv;
     }
-    
+
+    private void setupMyId() throws IOException {
+        File myIdFile = new File(dataDir, "myid");
+        // standalone server doesn't need myid file.
+        if (!myIdFile.isFile()) {
+            return;
+        }
+        BufferedReader br = new BufferedReader(new FileReader(myIdFile));
+        String myIdString;
+        try {
+            myIdString = br.readLine();
+        } finally {
+            br.close();
+        }
+        try {
+            serverId = Long.parseLong(myIdString);
+            MDC.put("myid", myIdString);
+        } catch (NumberFormatException e) {
+            throw new IllegalArgumentException("serverid " + myIdString
+                    + " is not a number");
+        }
+    }
+
+    private void setupClientPort() throws ConfigException {
+        if (serverId == UNSET_SERVERID) {
+            return;
+        }
+        QuorumServer qs = quorumVerifier.getAllMembers().get(serverId);
+        if (clientPortAddress != null && qs != null && qs.clientAddr != null) {
+            if ((!clientPortAddress.getAddress().isAnyLocalAddress()
+                    && !clientPortAddress.equals(qs.clientAddr)) ||
+                    (clientPortAddress.getAddress().isAnyLocalAddress()
+                            && clientPortAddress.getPort() != qs.clientAddr.getPort()))
+                throw new ConfigException("client address for this server (id = " + serverId +
+                        ") in static config file is " + clientPortAddress +
+                        " is different from client address found in dynamic file: " + qs.clientAddr);
+        }
+        if (qs != null && qs.clientAddr != null) clientPortAddress = qs.clientAddr;
+    }
+
+    private void setupPeerType() {
+        // Warn about inconsistent peer type
+        LearnerType roleByServersList = quorumVerifier.getObservingMembers().containsKey(serverId) ? LearnerType.OBSERVER
+                : LearnerType.PARTICIPANT;
+        if (roleByServersList != peerType) {
+            LOG.warn("Peer type from servers list (" + roleByServersList
+                    + ") doesn't match peerType (" + peerType
+                    + "). Defaulting to servers list.");
+
+            peerType = roleByServersList;
+        }
+    }
 
     public void checkValidity() throws IOException, ConfigException{
-       int numMembers = quorumVerifier.getVotingMembers().size();
-       if (numMembers > 1  || (!standaloneEnabled && numMembers > 0)) {
-           if (initLimit == 0) {
-               throw new IllegalArgumentException("initLimit is not set");
-           }
-           if (syncLimit == 0) {
-               throw new IllegalArgumentException("syncLimit is not set");
-           }
-            
-                                     
-            File myIdFile = new File(dataDir, "myid");
-            if (!myIdFile.exists()) {
-                throw new IllegalArgumentException(myIdFile.toString()
-                        + " file is missing");
+        if (isDistributed()) {
+            if (initLimit == 0) {
+                throw new IllegalArgumentException("initLimit is not set");
             }
-            BufferedReader br = new BufferedReader(new FileReader(myIdFile));
-            String myIdString;
-            try {
-                myIdString = br.readLine();
-            } finally {
-                br.close();
-            }
-            try {
-                serverId = Long.parseLong(myIdString);
-                MDC.put("myid", myIdString);
-            } catch (NumberFormatException e) {
-                throw new IllegalArgumentException("serverid " + myIdString
-                        + " is not a number");
-            }
-
-            QuorumServer qs = quorumVerifier.getAllMembers().get(serverId);
-            if (clientPortAddress!=null && qs!=null && qs.clientAddr!=null){ 
-                if ((!clientPortAddress.getAddress().isAnyLocalAddress()
-                       && !clientPortAddress.equals(qs.clientAddr)) || 
-                   (clientPortAddress.getAddress().isAnyLocalAddress() 
-                       && clientPortAddress.getPort()!=qs.clientAddr.getPort())) 
-                    throw new ConfigException("client address for this server (id = " + serverId + ") in static config file is " + clientPortAddress + " is different from client address found in dynamic file: " + qs.clientAddr);
+            if (syncLimit == 0) {
+                throw new IllegalArgumentException("syncLimit is not set");
             }
-            if (qs!=null && qs.clientAddr != null) clientPortAddress = qs.clientAddr;                       
-            
-            // Warn about inconsistent peer type
-            LearnerType roleByServersList = quorumVerifier.getObservingMembers().containsKey(serverId) ? LearnerType.OBSERVER
-                    : LearnerType.PARTICIPANT;
-            if (roleByServersList != peerType) {
-                LOG.warn("Peer type from servers list (" + roleByServersList
-                        + ") doesn't match peerType (" + peerType
-                        + "). Defaulting to servers list.");
-
-                peerType = roleByServersList;
+            if (serverId == UNSET_SERVERID) {
+                throw new IllegalArgumentException("myid file is missing");
             }
-           
        }
-
     }
-    
+
     public InetSocketAddress getClientPortAddress() { return clientPortAddress; }
     public File getDataDir() { return dataDir; }
     public File getDataLogDir() { return dataLogDir; }

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=1618977&r1=1618976&r2=1618977&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 Tue Aug 19 20:50:41 2014
@@ -66,8 +66,21 @@ public class QuorumPeerTestBase extends 
         final File confFile;
         final File tmpDir;
 
+        public static final int UNSET_STATIC_CLIENTPORT = -1;
+        // standalone mode doens't need myid
+        public static final int UNSET_MYID = -1;
+
         volatile TestQPMain main;
 
+        public MainThread(int myid, String quorumCfgSection) throws IOException {
+            this(myid, quorumCfgSection, true);
+        }
+
+        public MainThread(int myid, String quorumCfgSection, boolean writeDynamicConfigFile)
+                throws IOException {
+            this(myid, UNSET_STATIC_CLIENTPORT, quorumCfgSection, writeDynamicConfigFile);
+        }
+
         public MainThread(int myid, int clientPort, String quorumCfgSection)
                 throws IOException {
             this(myid, clientPort, JettyAdminServer.DEFAULT_PORT, quorumCfgSection, null, true);
@@ -126,11 +139,15 @@ public class QuorumPeerTestBase extends 
             String dir = PathUtils.normalizeFileSystemPath(dataDir.toString());
 
             fwriter.write("dataDir=" + dir + "\n");
-
-            fwriter.write("clientPort=" + clientPort + "\n");
-
             fwriter.write("admin.serverPort=" + adminServerPort + "\n");
 
+            // For backward compatibility test, some tests create dynamic configuration
+            // without setting client port.
+            // This could happen both in static file or dynamic file.
+            if (clientPort != UNSET_STATIC_CLIENTPORT) {
+                fwriter.write("clientPort=" + clientPort + "\n");
+            }
+
             if (writeDynamicConfigFile) {
                 String dynamicConfigFilename = createDynamicFile(quorumCfgSection, version);
                 fwriter.write("dynamicConfigFile=" + dynamicConfigFilename + "\n");

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/StandaloneTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/StandaloneTest.java?rev=1618977&r1=1618976&r2=1618977&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/StandaloneTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/StandaloneTest.java Tue Aug 19 20:50:41 2014
@@ -43,37 +43,75 @@ import org.junit.Test;
  */
 public class StandaloneTest extends QuorumPeerTestBase implements Watcher{
     protected static final Logger LOG =
-        LoggerFactory.getLogger(StandaloneTest.class);    
-      
+        LoggerFactory.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)
+     * This test wouldn't create any dynamic config.
+     * However, it adds a "clientPort=XXX" in static config file.
+     * It checks the standard way of standalone mode.
      */
     @Test
-    public void testStandaloneQuorum() throws Exception {
+    public void testNoDynamicConfig() throws Exception {
         ClientBase.setupTestEnv();
-        final int CLIENT_PORT_QP1 = PortAssignment.unique();        
-        
-        String quorumCfgSection =
-            "server.1=127.0.0.1:" + (PortAssignment.unique())
-            + ":" + (PortAssignment.unique()) + ";" + CLIENT_PORT_QP1 + "\n";
-                    
-        MainThread q1 = new MainThread(1, CLIENT_PORT_QP1, quorumCfgSection);
-        q1.start();
+        final int CLIENT_PORT = PortAssignment.unique();
+
+        MainThread mt = new MainThread(
+                MainThread.UNSET_MYID, CLIENT_PORT, "", false);
+        verifyStandalone(mt, CLIENT_PORT);
+    }
+
+    /**
+     * This test creates a dynamic config of new format.
+     * The dynamic config is written in dynamic config file.
+     * It checks that the client port will be read from the dynamic config.
+     *
+     * This handles the case of HBase, which adds a single server line to the config.
+     * Maintain b/w compatibility.
+     */
+    @Test
+    public void testClientPortInDynamicFile() throws Exception {
+        ClientBase.setupTestEnv();
+        final int CLIENT_PORT = PortAssignment.unique();
+
+        String quorumCfgSection = "server.1=127.0.0.1:" +
+                (PortAssignment.unique()) + ":" + (PortAssignment.unique())
+                + ":participant;" + CLIENT_PORT + "\n";
+
+        MainThread mt = new MainThread(1, quorumCfgSection);
+        verifyStandalone(mt, CLIENT_PORT);
+    }
+
+    /**
+     * This test creates a dynamic config of new format.
+     * The dynamic config is written in static config file.
+     * It checks that the client port will be read from the dynamic config.
+     */
+    @Test
+    public void testClientPortInStaticFile() throws Exception {
+        ClientBase.setupTestEnv();
+        final int CLIENT_PORT = PortAssignment.unique();
+
+        String quorumCfgSection = "server.1=127.0.0.1:" +
+                (PortAssignment.unique()) + ":" + (PortAssignment.unique())
+                + ":participant;" + CLIENT_PORT + "\n";
+
+        MainThread mt = new MainThread(1, quorumCfgSection, false);
+        verifyStandalone(mt, CLIENT_PORT);
+    }
+
+    void verifyStandalone(MainThread mt, int clientPort) throws InterruptedException {
+        mt.start();
         try {
             Assert.assertTrue("waiting for server 1 being up",
-                    ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1,
-                    CONNECTION_TIMEOUT));
-    } finally {
-        Assert.assertFalse("Error- MainThread started in Quorum Mode!",
-                   q1.isQuorumPeerRunning());
-        q1.shutdown();
+                    ClientBase.waitForServerUp("127.0.0.1:" + clientPort,
+                            CONNECTION_TIMEOUT));
+        } finally {
+            Assert.assertFalse("Error- MainThread started in Quorum Mode!",
+                    mt.isQuorumPeerRunning());
+            mt.shutdown();
         }
-    }    
-    
+    }
+
     /**
      * Verify that reconfiguration in standalone mode fails with
      * KeeperException.UnimplementedException.