You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by mi...@apache.org on 2014/01/21 04:50:01 UTC
svn commit: r1559916 - in /zookeeper/trunk: ./
src/docs/src/documentation/content/xdocs/
src/java/main/org/apache/zookeeper/cli/
src/java/main/org/apache/zookeeper/server/
src/java/main/org/apache/zookeeper/server/quorum/
src/java/test/org/apache/zooke...
Author: michim
Date: Tue Jan 21 03:50:01 2014
New Revision: 1559916
URL: http://svn.apache.org/r1559916
Log:
ZOOKEEPER-1691. Add a flag to disable standalone mode (Helen Hastings via michim)
Modified:
zookeeper/trunk/CHANGES.txt
zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/ReconfigCommand.java
zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java
zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.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/quorum/QuorumPeerTestBase.java
zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ReconfigTest.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=1559916&r1=1559915&r2=1559916&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Tue Jan 21 03:50:01 2014
@@ -18,6 +18,8 @@ NEW FEATURES:
ZOOKEEPER-1147. Add support for local sessions (Jay Shrauner, thawan via thawan)
+ ZOOKEEPER-1691. Add a flag to disable standalone mode (Helen Hastings via michim)
+
BUGFIXES:
ZOOKEEPER-786. Exception in ZooKeeper.toString
Modified: zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml?rev=1559916&r1=1559915&r2=1559916&view=diff
==============================================================================
--- zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml (original)
+++ zookeeper/trunk/src/docs/src/documentation/content/xdocs/zookeeperAdmin.xml Tue Jan 21 03:50:01 2014
@@ -1000,6 +1000,24 @@ server.3=zoo3:2888:3888</programlisting>
</note>
</listitem>
</varlistentry>
+
+ <varlistentry>
+ <term>standaloneEnabled</term>
+
+ <listitem>
+ <para>(No Java system property)</para>
+
+ <para><emphasis role="bold">New in 3.5.0:</emphasis>
+ When set to false, a single server can be started in replicated
+ mode, a lone participant can run with observers, and a cluster
+ can reconfigure down to one node, and up from one node. The
+ default is true for backwards compatibility. It can be set
+ using QuorumPeerConfig's setStandaloneEnabled method or by
+ adding "standaloneEnabled=false" or "standaloneEnabled=true"
+ to a server's config file.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist>
<para></para>
</section>
Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/ReconfigCommand.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/ReconfigCommand.java?rev=1559916&r1=1559915&r2=1559916&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/ReconfigCommand.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/cli/ReconfigCommand.java Tue Jan 21 03:50:01 2014
@@ -121,7 +121,6 @@ public class ReconfigCommand extends Cli
if (cl.hasOption("file")) {
try {
FileInputStream inConfig = new FileInputStream(cl.getOptionValue("file"));
- QuorumPeerConfig config = new QuorumPeerConfig();
Properties dynamicCfg = new Properties();
try {
dynamicCfg.load(inConfig);
@@ -131,8 +130,7 @@ public class ReconfigCommand extends Cli
//check that membership makes sense; leader will make these checks again
//don't check for leader election ports since
//client doesn't know what leader election alg is used
- config.parseDynamicConfig(dynamicCfg, 0, true);
- members = config.getQuorumVerifier().toString();
+ members = QuorumPeerConfig.parseDynamicConfig(dynamicCfg, 0, true, false).toString();
} catch (Exception e) {
throw new ParseException("Error processing " + cl.getOptionValue("file") + e.getMessage());
}
Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java?rev=1559916&r1=1559915&r2=1559916&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/PrepRequestProcessor.java Tue Jan 21 03:50:01 2014
@@ -539,9 +539,7 @@ public class PrepRequestProcessor extend
try{
Properties props = new Properties();
props.load(new StringReader(newMembers));
- QuorumPeerConfig config = new QuorumPeerConfig();
- config.parseDynamicConfig(props, lzks.self.getElectionType(), true);
- request.qv = config.getQuorumVerifier();
+ request.qv = QuorumPeerConfig.parseDynamicConfig(props, lzks.self.getElectionType(), true, false);
request.qv.setVersion(request.getHdr().getZxid());
} catch (IOException e) {
throw new KeeperException.BadArgumentsException(e.getMessage());
@@ -600,12 +598,16 @@ public class PrepRequestProcessor extend
}
request.qv = new QuorumMaj(nextServers);
request.qv.setVersion(request.getHdr().getZxid());
- }
- if (request.qv.getVotingMembers().size() < 2){
+ }
+ if (QuorumPeerConfig.isStandaloneEnabled() && request.qv.getVotingMembers().size() < 2) {
String msg = "Reconfig failed - new configuration must include at least 2 followers";
LOG.warn(msg);
throw new KeeperException.BadArgumentsException(msg);
- }
+ } else if (request.qv.getVotingMembers().size() < 1) {
+ String msg = "Reconfig failed - new configuration must include at least 1 follower";
+ LOG.warn(msg);
+ throw new KeeperException.BadArgumentsException(msg);
+ }
if (!lzks.getLeader().isQuorumSynced(request.qv)) {
String msg2 = "Reconfig failed - there must be a connected and synced quorum in new configuration";
Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java?rev=1559916&r1=1559915&r2=1559916&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/FollowerZooKeeperServer.java Tue Jan 21 03:50:01 2014
@@ -127,7 +127,8 @@ public class FollowerZooKeeperServer ext
@Override
public int getGlobalOutstandingLimit() {
- return super.getGlobalOutstandingLimit() / (self.getQuorumSize() - 1);
+ int divisor = self.getQuorumSize() > 2 ? self.getQuorumSize() - 1 : 1;
+ return super.getGlobalOutstandingLimit() / divisor;
}
@Override
Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java?rev=1559916&r1=1559915&r2=1559916&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/quorum/LeaderZooKeeperServer.java Tue Jan 21 03:50:01 2014
@@ -74,7 +74,8 @@ public class LeaderZooKeeperServer exten
@Override
public int getGlobalOutstandingLimit() {
- return super.getGlobalOutstandingLimit() / (self.getQuorumSize() - 1);
+ int divisor = self.getQuorumSize() > 2 ? self.getQuorumSize() - 1 : 1;
+ return super.getGlobalOutstandingLimit() / divisor;
}
@Override
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=1559916&r1=1559915&r2=1559916&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 Tue Jan 21 03:50:01 2014
@@ -1228,11 +1228,7 @@ public class QuorumPeer extends Thread i
public QuorumVerifier configFromString(String s) throws IOException, ConfigException{
Properties props = new Properties();
props.load(new StringReader(s));
-
- QuorumPeerConfig config = new QuorumPeerConfig();
- config.parseDynamicConfig(props, electionType, false);
-
- return config.getQuorumVerifier();
+ return QuorumPeerConfig.parseDynamicConfig(props, electionType, false, false);
}
/**
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=1559916&r1=1559915&r2=1559916&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 Jan 21 03:50:01 2014
@@ -49,6 +49,7 @@ import org.apache.zookeeper.server.util.
public class QuorumPeerConfig {
private static final Logger LOG = LoggerFactory.getLogger(QuorumPeerConfig.class);
+ private static boolean standaloneEnabled = true;
protected InetSocketAddress clientPortAddress;
protected File dataDir;
@@ -125,7 +126,7 @@ public class QuorumPeerConfig {
if (dynamicConfigFileStr == null) {
configBackwardCompatibilityMode = true;
configFileStr = path;
- parseDynamicConfig(cfg, electionAlg, true);
+ quorumVerifier = parseDynamicConfig(cfg, electionAlg, true, configBackwardCompatibilityMode);
checkValidity();
}
@@ -144,7 +145,7 @@ public class QuorumPeerConfig {
} finally {
inConfig.close();
}
- parseDynamicConfig(dynamicCfg, electionAlg, true);
+ quorumVerifier = parseDynamicConfig(dynamicCfg, electionAlg, true, configBackwardCompatibilityMode);
checkValidity();
} catch (IOException e) {
@@ -237,8 +238,16 @@ public class QuorumPeerConfig {
snapRetainCount = Integer.parseInt(value);
} else if (key.equals("autopurge.purgeInterval")) {
purgeInterval = Integer.parseInt(value);
- } else if ((key.startsWith("server.") || key.startsWith("group") || key.startsWith("weight")) && zkProp.containsKey("dynamicConfigFile")){
- throw new ConfigException("parameter: " + key + " must be in a separate dynamic config file");
+ } else if (key.equals("standaloneEnabled")) {
+ if (value.toLowerCase().equals("true")) {
+ setStandaloneEnabled(true);
+ } else if (value.toLowerCase().equals("false")) {
+ setStandaloneEnabled(false);
+ } else {
+ throw new ConfigException("Invalid option for standalone mode. Choose 'true' or 'false.'");
+ }
+ } else if ((key.startsWith("server.") || key.startsWith("group") || key.startsWith("weight")) && zkProp.containsKey("dynamicConfigFile")) {
+ throw new ConfigException("parameter: " + key + " must be in a separate dynamic config file");
} else {
System.setProperty("zookeeper." + key, value);
}
@@ -370,7 +379,7 @@ public class QuorumPeerConfig {
}
- private QuorumVerifier createQuorumVerifier(Properties dynamicConfigProp, boolean isHierarchical) throws ConfigException{
+ private static QuorumVerifier createQuorumVerifier(Properties dynamicConfigProp, boolean isHierarchical) throws ConfigException{
if(isHierarchical){
return new QuorumHierarchical(dynamicConfigProp);
} else {
@@ -383,13 +392,14 @@ public class QuorumPeerConfig {
}
/**
- * Parse dynamic configuration file.
+ * Parse dynamic configuration file and return
+ * quorumVerifier for new configuration.
* @param zkProp Properties to parse from.
* @throws IOException
* @throws ConfigException
*/
- public void parseDynamicConfig(Properties dynamicConfigProp, int eAlg, boolean warnings)
- throws IOException, ConfigException {
+ public static QuorumVerifier parseDynamicConfig(Properties dynamicConfigProp, int eAlg, boolean warnings,
+ boolean configBackwardCompatibilityMode) throws IOException, ConfigException {
boolean isHierarchical = false;
for (Entry<Object, Object> entry : dynamicConfigProp.entrySet()) {
String key = entry.getKey().toString().trim();
@@ -401,54 +411,51 @@ public class QuorumPeerConfig {
}
}
- quorumVerifier = createQuorumVerifier(dynamicConfigProp, isHierarchical);
+ QuorumVerifier qv = createQuorumVerifier(dynamicConfigProp, isHierarchical);
- int numParticipators = quorumVerifier.getVotingMembers().size();
- int numObservers = quorumVerifier.getObservingMembers().size();
+ int numParticipators = qv.getVotingMembers().size();
+ int numObservers = qv.getObservingMembers().size();
if (numParticipators == 0) {
if (numObservers > 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 (numParticipators == 1) {
+ } else if (numParticipators == 1 && standaloneEnabled) {
+ // HBase currently adds a single server line to the config, for
+ // b/w compatibility reasons we need to keep this here. If standaloneEnabled
+ // is true, the QuorumPeerMain script will create a standalone server instead
+ // of a quorum configuration
+ LOG.error("Invalid configuration, only one server specified (ignoring)");
if (numObservers > 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 (numParticipators > 1) {
- if (warnings) {
- if (numParticipators == 2) {
+ } else {
+ if (warnings) {
+ if (numParticipators <= 2) {
LOG.warn("No server failure will be tolerated. " +
"You need at least 3 servers.");
} else if (numParticipators % 2 == 0) {
LOG.warn("Non-optimial configuration, consider an odd number of servers.");
}
- }
+ }
/*
* If using FLE, then every server requires a separate election
* port.
*/
if (eAlg != 0) {
- for (QuorumServer s : quorumVerifier.getVotingMembers().values()) {
+ for (QuorumServer s : qv.getVotingMembers().values()) {
if (s.electionAddr == null)
throw new IllegalArgumentException(
"Missing election port for server: " + s.id);
}
}
}
+ return qv;
}
public void checkValidity() throws IOException, ConfigException{
-
- if (quorumVerifier.getVotingMembers().size() > 1) {
+ int numMembers = quorumVerifier.getVotingMembers().size();
+ if (numMembers > 1 || (!standaloneEnabled && numMembers > 0)) {
if (initLimit == 0) {
throw new IllegalArgumentException("initLimit is not set");
}
@@ -546,7 +553,9 @@ public class QuorumPeerConfig {
public long getServerId() { return serverId; }
- public boolean isDistributed() { return (quorumVerifier!=null && quorumVerifier.getVotingMembers().size() > 1); }
+ public boolean isDistributed() {
+ return quorumVerifier!=null && (!standaloneEnabled || quorumVerifier.getVotingMembers().size() > 1);
+ }
public LearnerType getPeerType() {
return peerType;
@@ -567,5 +576,13 @@ public class QuorumPeerConfig {
public Boolean getQuorumListenOnAllIPs() {
return quorumListenOnAllIPs;
}
+
+ public static boolean isStandaloneEnabled() {
+ return standaloneEnabled;
+ }
+
+ public static void setStandaloneEnabled(boolean enabled) {
+ standaloneEnabled = enabled;
+ }
}
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=1559916&r1=1559915&r2=1559916&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 Jan 21 03:50:01 2014
@@ -170,5 +170,9 @@ public class QuorumPeerTestBase extends
ClientBase.recursiveDelete(main.quorumPeer.getTxnFactory()
.getDataDir());
}
+
+ public boolean isQuorumPeerRunning() {
+ return main.quorumPeer != null;
+ }
}
}
Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ReconfigTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ReconfigTest.java?rev=1559916&r1=1559915&r2=1559916&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ReconfigTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ReconfigTest.java Tue Jan 21 03:50:01 2014
@@ -47,7 +47,7 @@ public class ReconfigTest extends ZKTest
private static final Logger LOG = LoggerFactory
.getLogger(ReconfigTest.class);
- private String reconfig(ZooKeeper zk, List<String> joiningServers,
+ public static String reconfig(ZooKeeper zk, List<String> joiningServers,
List<String> leavingServers, List<String> newMembers, long fromConfig)
throws KeeperException, InterruptedException {
byte[] config = null;
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=1559916&r1=1559915&r2=1559916&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 Jan 21 03:50:01 2014
@@ -27,6 +27,7 @@ import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.server.quorum.QuorumPeerTestBase;
import org.junit.Assert;
import org.junit.Test;
+import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
/**
* Standalone server tests.
@@ -56,9 +57,11 @@ public class StandaloneTest extends Quor
try {
Assert.assertTrue("waiting for server 1 being up",
ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT_QP1,
- CONNECTION_TIMEOUT));
- } finally {
- q1.shutdown();
+ CONNECTION_TIMEOUT));
+ } finally {
+ Assert.assertFalse("Error- MainThread started in Quorum Mode!",
+ q1.isQuorumPeerRunning());
+ q1.shutdown();
}
}