|
@@ -20,6 +20,8 @@ package org.apache.zookeeper.server;
|
|
|
|
|
|
import java.util.Random;
|
|
|
import org.apache.zookeeper.common.Time;
|
|
|
+import org.slf4j.Logger;
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
|
|
|
/**
|
|
|
* Implements a token-bucket based rate limiting mechanism with optional
|
|
@@ -69,6 +71,7 @@ import org.apache.zookeeper.common.Time;
|
|
|
**/
|
|
|
|
|
|
public class BlueThrottle {
|
|
|
+ private static final Logger LOG = LoggerFactory.getLogger(BlueThrottle.class);
|
|
|
|
|
|
private int maxTokens;
|
|
|
private int fillTime;
|
|
@@ -86,35 +89,115 @@ public class BlueThrottle {
|
|
|
Random rng;
|
|
|
|
|
|
public static final String CONNECTION_THROTTLE_TOKENS = "zookeeper.connection_throttle_tokens";
|
|
|
- public static final int DEFAULT_CONNECTION_THROTTLE_TOKENS;
|
|
|
+ private static final int DEFAULT_CONNECTION_THROTTLE_TOKENS;
|
|
|
|
|
|
public static final String CONNECTION_THROTTLE_FILL_TIME = "zookeeper.connection_throttle_fill_time";
|
|
|
- public static final int DEFAULT_CONNECTION_THROTTLE_FILL_TIME;
|
|
|
+ private static final int DEFAULT_CONNECTION_THROTTLE_FILL_TIME;
|
|
|
|
|
|
public static final String CONNECTION_THROTTLE_FILL_COUNT = "zookeeper.connection_throttle_fill_count";
|
|
|
- public static final int DEFAULT_CONNECTION_THROTTLE_FILL_COUNT;
|
|
|
+ private static final int DEFAULT_CONNECTION_THROTTLE_FILL_COUNT;
|
|
|
|
|
|
public static final String CONNECTION_THROTTLE_FREEZE_TIME = "zookeeper.connection_throttle_freeze_time";
|
|
|
- public static final int DEFAULT_CONNECTION_THROTTLE_FREEZE_TIME;
|
|
|
+ private static final int DEFAULT_CONNECTION_THROTTLE_FREEZE_TIME;
|
|
|
|
|
|
public static final String CONNECTION_THROTTLE_DROP_INCREASE = "zookeeper.connection_throttle_drop_increase";
|
|
|
- public static final double DEFAULT_CONNECTION_THROTTLE_DROP_INCREASE;
|
|
|
+ private static final double DEFAULT_CONNECTION_THROTTLE_DROP_INCREASE;
|
|
|
|
|
|
public static final String CONNECTION_THROTTLE_DROP_DECREASE = "zookeeper.connection_throttle_drop_decrease";
|
|
|
- public static final double DEFAULT_CONNECTION_THROTTLE_DROP_DECREASE;
|
|
|
+ private static final double DEFAULT_CONNECTION_THROTTLE_DROP_DECREASE;
|
|
|
|
|
|
public static final String CONNECTION_THROTTLE_DECREASE_RATIO = "zookeeper.connection_throttle_decrease_ratio";
|
|
|
- public static final double DEFAULT_CONNECTION_THROTTLE_DECREASE_RATIO;
|
|
|
+ private static final double DEFAULT_CONNECTION_THROTTLE_DECREASE_RATIO;
|
|
|
+
|
|
|
+ public static final String WEIGHED_CONNECTION_THROTTLE = "zookeeper.connection_throttle_weight_enabled";
|
|
|
+ private static boolean connectionWeightEnabled;
|
|
|
+
|
|
|
+ public static final String GLOBAL_SESSION_WEIGHT = "zookeeper.connection_throttle_global_session_weight";
|
|
|
+ private static final int DEFAULT_GLOBAL_SESSION_WEIGHT;
|
|
|
+
|
|
|
+ public static final String LOCAL_SESSION_WEIGHT = "zookeeper.connection_throttle_local_session_weight";
|
|
|
+ private static final int DEFAULT_LOCAL_SESSION_WEIGHT;
|
|
|
+
|
|
|
+ public static final String RENEW_SESSION_WEIGHT = "zookeeper.connection_throttle_renew_session_weight";
|
|
|
+ private static final int DEFAULT_RENEW_SESSION_WEIGHT;
|
|
|
+
|
|
|
+ // for unit tests only
|
|
|
+ protected static void setConnectionWeightEnabled(boolean enabled) {
|
|
|
+ connectionWeightEnabled = enabled;
|
|
|
+ logWeighedThrottlingSetting();
|
|
|
+ }
|
|
|
+
|
|
|
+ private static void logWeighedThrottlingSetting() {
|
|
|
+ if (connectionWeightEnabled) {
|
|
|
+ LOG.info("Weighed connection throttling is enabled. "
|
|
|
+ + "But it will only be effective if connection throttling is enabled");
|
|
|
+ LOG.info(
|
|
|
+ "The weights for different session types are: global {} renew {} local {}",
|
|
|
+ DEFAULT_GLOBAL_SESSION_WEIGHT,
|
|
|
+ DEFAULT_RENEW_SESSION_WEIGHT,
|
|
|
+ DEFAULT_LOCAL_SESSION_WEIGHT
|
|
|
+ );
|
|
|
+ } else {
|
|
|
+ LOG.info("Weighed connection throttling is disabled");
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
static {
|
|
|
- DEFAULT_CONNECTION_THROTTLE_TOKENS = Integer.getInteger(CONNECTION_THROTTLE_TOKENS, 0);
|
|
|
- DEFAULT_CONNECTION_THROTTLE_FILL_TIME = Integer.getInteger(CONNECTION_THROTTLE_FILL_TIME, 1);
|
|
|
- DEFAULT_CONNECTION_THROTTLE_FILL_COUNT = Integer.getInteger(CONNECTION_THROTTLE_FILL_COUNT, 1);
|
|
|
+ int tokens = Integer.getInteger(CONNECTION_THROTTLE_TOKENS, 0);
|
|
|
+ int fillCount = Integer.getInteger(CONNECTION_THROTTLE_FILL_COUNT, 1);
|
|
|
+
|
|
|
+ connectionWeightEnabled = Boolean.getBoolean(WEIGHED_CONNECTION_THROTTLE);
|
|
|
+
|
|
|
+ // if not specified, the weights for a global session, a local session, and a renew session
|
|
|
+ // are 3, 1, 2 respectively. The weight for a global session is 3 because in our connection benchmarking,
|
|
|
+ // the throughput of global sessions is about one third of that of local sessions. Renewing a session
|
|
|
+ // requires is more expensive than establishing a local session and cheaper than creating a global session so
|
|
|
+ // its default weight is set to 2.
|
|
|
+ int globalWeight = Integer.getInteger(GLOBAL_SESSION_WEIGHT, 3);
|
|
|
+ int localWeight = Integer.getInteger(LOCAL_SESSION_WEIGHT, 1);
|
|
|
+ int renewWeight = Integer.getInteger(RENEW_SESSION_WEIGHT, 2);
|
|
|
+
|
|
|
+ if (globalWeight <= 0) {
|
|
|
+ LOG.warn("Invalid global session weight {}. It should be larger than 0", globalWeight);
|
|
|
+ DEFAULT_GLOBAL_SESSION_WEIGHT = 3;
|
|
|
+ } else if (globalWeight < localWeight) {
|
|
|
+ LOG.warn("The global session weight {} is less than the local session weight {}. Use the local session weight.",
|
|
|
+ globalWeight, localWeight);
|
|
|
+ DEFAULT_GLOBAL_SESSION_WEIGHT = localWeight;
|
|
|
+ } else {
|
|
|
+ DEFAULT_GLOBAL_SESSION_WEIGHT = globalWeight;
|
|
|
+ }
|
|
|
|
|
|
+ if (localWeight <= 0) {
|
|
|
+ LOG.warn("Invalid local session weight {}. It should be larger than 0", localWeight);
|
|
|
+ DEFAULT_LOCAL_SESSION_WEIGHT = 1;
|
|
|
+ } else {
|
|
|
+ DEFAULT_LOCAL_SESSION_WEIGHT = localWeight;
|
|
|
+ }
|
|
|
+
|
|
|
+ if (renewWeight <= 0) {
|
|
|
+ LOG.warn("Invalid renew session weight {}. It should be larger than 0", renewWeight);
|
|
|
+ DEFAULT_RENEW_SESSION_WEIGHT = 2;
|
|
|
+ } else if (renewWeight < localWeight) {
|
|
|
+ LOG.warn("The renew session weight {} is less than the local session weight {}. Use the local session weight.",
|
|
|
+ renewWeight, localWeight);
|
|
|
+ DEFAULT_RENEW_SESSION_WEIGHT = localWeight;
|
|
|
+ } else {
|
|
|
+ DEFAULT_RENEW_SESSION_WEIGHT = renewWeight;
|
|
|
+ }
|
|
|
+
|
|
|
+ // This is based on the assumption that tokens set in config are for global sessions
|
|
|
+ DEFAULT_CONNECTION_THROTTLE_TOKENS = connectionWeightEnabled
|
|
|
+ ? DEFAULT_GLOBAL_SESSION_WEIGHT * tokens : tokens;
|
|
|
+ DEFAULT_CONNECTION_THROTTLE_FILL_TIME = Integer.getInteger(CONNECTION_THROTTLE_FILL_TIME, 1);
|
|
|
+ DEFAULT_CONNECTION_THROTTLE_FILL_COUNT = connectionWeightEnabled
|
|
|
+ ? DEFAULT_GLOBAL_SESSION_WEIGHT * fillCount : fillCount;
|
|
|
DEFAULT_CONNECTION_THROTTLE_FREEZE_TIME = Integer.getInteger(CONNECTION_THROTTLE_FREEZE_TIME, -1);
|
|
|
DEFAULT_CONNECTION_THROTTLE_DROP_INCREASE = getDoubleProp(CONNECTION_THROTTLE_DROP_INCREASE, 0.02);
|
|
|
DEFAULT_CONNECTION_THROTTLE_DROP_DECREASE = getDoubleProp(CONNECTION_THROTTLE_DROP_DECREASE, 0.002);
|
|
|
DEFAULT_CONNECTION_THROTTLE_DECREASE_RATIO = getDoubleProp(CONNECTION_THROTTLE_DECREASE_RATIO, 0);
|
|
|
+
|
|
|
+ logWeighedThrottlingSetting();
|
|
|
}
|
|
|
|
|
|
/* Varation of Integer.getInteger for real number properties */
|
|
@@ -212,6 +295,22 @@ public class BlueThrottle {
|
|
|
return maxTokens - tokens;
|
|
|
}
|
|
|
|
|
|
+ public int getRequiredTokensForGlobal() {
|
|
|
+ return BlueThrottle.DEFAULT_GLOBAL_SESSION_WEIGHT;
|
|
|
+ }
|
|
|
+
|
|
|
+ public int getRequiredTokensForLocal() {
|
|
|
+ return BlueThrottle.DEFAULT_LOCAL_SESSION_WEIGHT;
|
|
|
+ }
|
|
|
+
|
|
|
+ public int getRequiredTokensForRenew() {
|
|
|
+ return BlueThrottle.DEFAULT_RENEW_SESSION_WEIGHT;
|
|
|
+ }
|
|
|
+
|
|
|
+ public boolean isConnectionWeightEnabled() {
|
|
|
+ return BlueThrottle.connectionWeightEnabled;
|
|
|
+ }
|
|
|
+
|
|
|
public synchronized boolean checkLimit(int need) {
|
|
|
// A maxTokens setting of zero disables throttling
|
|
|
if (maxTokens == 0) {
|