Skip to content

Commit 6433cbd

Browse files
Limit the number of ephemeral nodes a session can create (#118)
This throttles ephemeral node creation if the total size of all characters of ephemeral node paths exceeds threshold.
1 parent a9a109f commit 6433cbd

File tree

9 files changed

+384
-6
lines changed

9 files changed

+384
-6
lines changed

zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1070,6 +1070,13 @@ property, when available, is noted below.
10701070
**New in 3.6.0:**
10711071
The size threshold after which a request is considered a large request. If it is -1, then all requests are considered small, effectively turning off large request throttling. The default is -1.
10721072

1073+
* *ephemeralNodes.total.byte.limit* :
1074+
(Java system property: **zookeeper.ephemeralNodes.total.byte.limit**)
1075+
This property set a limit on the amount of ephemeral nodes that can be created in one session. The limit is the number
1076+
of bytes it takes to store the serialized path strings for all the session's ephemeral nodes.
1077+
This limit should always be under the jute maxbuffer, as exceeding will cause the server to crash when the connection is closed
1078+
and a transaction to delete all the ephemeral nodes for that session are deleted. This limit will be ignored if not explicitly set.
1079+
10731080
* *outstandingHandshake.limit*
10741081
(Jave system property only: **zookeeper.netty.server.outstandingHandshake.limit**)
10751082
The maximum in-flight TLS handshake connections could have in ZooKeeper,

zookeeper-jute/src/main/java/org/apache/jute/BinaryOutputArchive.java

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -102,6 +102,29 @@ private ByteBuffer stringToByteBuffer(CharSequence s) {
102102
return bb;
103103
}
104104

105+
public static int getSerializedStringByteSize(String s) throws ArithmeticException {
106+
if (s == null) {
107+
return 0;
108+
}
109+
110+
// Always add 4 bytes to size as we call writeInt(bb.remaining(), "len") when writing to DataOutput
111+
int length_descriptor_size = 4;
112+
113+
int size = 0;
114+
final int len = s.length();
115+
for (int i = 0; i < len; i++) {
116+
char c = s.charAt(i);
117+
if (c < 0x80) {
118+
size = Math.addExact(size, 1);
119+
} else if (c < 0x800) {
120+
size = Math.addExact(size, 2);
121+
} else {
122+
size = Math.addExact(size, 3);
123+
}
124+
}
125+
return Math.addExact(size, length_descriptor_size);
126+
}
127+
105128
public void writeString(String s, String tag) throws IOException {
106129
if (s == null) {
107130
writeInt(-1, "len");

zookeeper-server/src/main/java/org/apache/zookeeper/KeeperException.java

Lines changed: 14 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -148,6 +148,8 @@ public static KeeperException create(Code code) {
148148
return new SessionClosedRequireAuthException();
149149
case REQUESTTIMEOUT:
150150
return new RequestTimeoutException();
151+
case TOTALEPHEMERALLIMITEXCEEDED:
152+
return new TotalEphemeralLimitExceeded();
151153
case OK:
152154
default:
153155
throw new IllegalArgumentException("Invalid exception code");
@@ -404,7 +406,10 @@ public enum Code implements CodeDeprecated {
404406
/** The session has been closed by server because server requires client to do SASL authentication,
405407
* but client is not configured with SASL authentication or configuted with SASL but failed
406408
* (i.e. wrong credential used.). */
407-
SESSIONCLOSEDREQUIRESASLAUTH(-124);
409+
SESSIONCLOSEDREQUIRESASLAUTH(-124),
410+
/** Request to create ephemeral node was rejected because the total byte limit for the session was exceeded.
411+
* This limit is manually set through the "zookeeper.ephemeralNodes.total.byte.limit" system property. */
412+
TOTALEPHEMERALLIMITEXCEEDED(-125);
408413

409414
private static final Map<Integer, Code> lookup = new HashMap<Integer, Code>();
410415

@@ -495,6 +500,8 @@ static String getCodeMessage(Code code) {
495500
return "Reconfig is disabled";
496501
case SESSIONCLOSEDREQUIRESASLAUTH:
497502
return "Session closed because client failed to authenticate";
503+
case TOTALEPHEMERALLIMITEXCEEDED:
504+
return "Ephemeral count exceeded for session";
498505
default:
499506
return "Unknown error " + code;
500507
}
@@ -940,4 +947,10 @@ public RequestTimeoutException() {
940947

941948
}
942949

950+
public static class TotalEphemeralLimitExceeded extends KeeperException {
951+
public TotalEphemeralLimitExceeded() {
952+
super(Code.TOTALEPHEMERALLIMITEXCEEDED);
953+
}
954+
}
955+
943956
}

zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java

Lines changed: 26 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -35,11 +35,10 @@
3535
import java.util.Map.Entry;
3636
import java.util.Set;
3737
import java.util.concurrent.ConcurrentHashMap;
38+
import java.util.concurrent.atomic.AtomicInteger;
3839
import java.util.concurrent.atomic.AtomicLong;
39-
import org.apache.jute.BinaryInputArchive;
40-
import org.apache.jute.InputArchive;
41-
import org.apache.jute.OutputArchive;
42-
import org.apache.jute.Record;
40+
41+
import org.apache.jute.*;
4342
import org.apache.zookeeper.DigestWatcher;
4443
import org.apache.zookeeper.KeeperException;
4544
import org.apache.zookeeper.KeeperException.Code;
@@ -160,6 +159,7 @@ public class DataTree {
160159
* This hashtable lists the paths of the ephemeral nodes of a session.
161160
*/
162161
private final Map<Long, HashSet<String>> ephemerals = new ConcurrentHashMap<Long, HashSet<String>>();
162+
private final Map<Long, AtomicInteger> ephemeralsByteSizeMap = new ConcurrentHashMap<Long, AtomicInteger>();
163163

164164
/**
165165
* This set contains the paths of all container nodes
@@ -222,6 +222,11 @@ public Set<String> getEphemerals(long sessionId) {
222222
return cloned;
223223
}
224224

225+
public int getTotalEphemeralsByteSize(long sessionID) {
226+
AtomicInteger byteSize = ephemeralsByteSizeMap.get(sessionID);
227+
return byteSize != null ? byteSize.get() : 0;
228+
}
229+
225230
public Set<String> getContainers() {
226231
return new HashSet<String>(containers);
227232
}
@@ -554,13 +559,22 @@ public void createNode(final String path, byte[] data, List<ACL> acl, long ephem
554559
ttls.add(path);
555560
} else if (ephemeralOwner != 0) {
556561
HashSet<String> list = ephemerals.get(ephemeralOwner);
562+
AtomicInteger totalEphemeralsByteSize = ephemeralsByteSizeMap.get(ephemeralOwner);
557563
if (list == null) {
558564
list = new HashSet<String>();
559565
ephemerals.put(ephemeralOwner, list);
560566
}
561567
synchronized (list) {
562568
list.add(path);
563569
}
570+
// Only store sum of ephemeral node byte sizes if we're enforcing a limit
571+
if (ZooKeeperServer.getEphemeralNodesTotalByteLimit() != -1) {
572+
if (totalEphemeralsByteSize == null) {
573+
totalEphemeralsByteSize = new AtomicInteger();
574+
ephemeralsByteSizeMap.put(ephemeralOwner, totalEphemeralsByteSize);
575+
}
576+
totalEphemeralsByteSize.addAndGet(BinaryOutputArchive.getSerializedStringByteSize(path));
577+
}
564578
}
565579
if (outputStat != null) {
566580
child.copyStat(outputStat);
@@ -645,9 +659,15 @@ public void deleteNode(String path, long zxid) throws KeeperException.NoNodeExce
645659
ttls.remove(path);
646660
} else if (eowner != 0) {
647661
Set<String> nodes = ephemerals.get(eowner);
662+
AtomicInteger totalEphemeralsByteSize = ephemeralsByteSizeMap.get(eowner);
648663
if (nodes != null) {
664+
Boolean nodeExisted;
649665
synchronized (nodes) {
650-
nodes.remove(path);
666+
nodeExisted = nodes.remove(path);
667+
}
668+
// Only store sum of ephemeral node byte sizes if we're enforcing a limit
669+
if (ZooKeeperServer.getEphemeralNodesTotalByteLimit() != -1 && nodeExisted && totalEphemeralsByteSize != null) {
670+
totalEphemeralsByteSize.addAndGet(-(BinaryOutputArchive.getSerializedStringByteSize(path)));
651671
}
652672
}
653673
}
@@ -1206,6 +1226,7 @@ public ProcessTxnResult processTxn(TxnHeader header, Record txn, boolean isSubTx
12061226
case OpCode.closeSession:
12071227
long sessionId = header.getClientId();
12081228
if (txn != null) {
1229+
ephemeralsByteSizeMap.remove(sessionId);
12091230
killSession(sessionId, header.getZxid(),
12101231
ephemerals.remove(sessionId),
12111232
((CloseSessionTxn) txn).getPaths2Delete());

zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import java.io.IOException;
2323
import java.io.StringReader;
2424
import java.nio.ByteBuffer;
25+
import java.nio.charset.StandardCharsets;
2526
import java.util.ArrayList;
2627
import java.util.Collections;
2728
import java.util.HashMap;
@@ -708,6 +709,7 @@ private void pRequest2TxnCreate(int type, Request request, Record record, boolea
708709
request.setTxn(new CreateTxn(path, data, listACL, createMode.isEphemeral(), newCversion));
709710
}
710711

712+
711713
TxnHeader hdr = request.getHdr();
712714
long ephemeralOwner = 0;
713715
if (createMode.isContainer()) {
@@ -716,6 +718,12 @@ private void pRequest2TxnCreate(int type, Request request, Record record, boolea
716718
ephemeralOwner = EphemeralType.TTL.toEphemeralOwner(ttl);
717719
} else if (createMode.isEphemeral()) {
718720
ephemeralOwner = request.sessionId;
721+
int currentByteSize = zks.getZKDatabase().getDataTree().getTotalEphemeralsByteSize(ephemeralOwner);
722+
if (ZooKeeperServer.getEphemeralNodesTotalByteLimit() != -1 && currentByteSize + BinaryOutputArchive.getSerializedStringByteSize(path)
723+
> ZooKeeperServer.getEphemeralNodesTotalByteLimit()) {
724+
ServerMetrics.getMetrics().EPHEMERAL_NODE_LIMIT_VIOLATION.inc();
725+
throw new KeeperException.TotalEphemeralLimitExceeded();
726+
}
719727
}
720728
StatPersisted s = DataTree.createStat(hdr.getZxid(), hdr.getTime(), ephemeralOwner);
721729
parentRecord = parentRecord.duplicate(request.getHdr().getZxid());

zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerMetrics.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -156,6 +156,7 @@ private ServerMetrics(MetricsProvider metricsProvider) {
156156
COMMITS_QUEUED = metricsContext.getCounter("request_commit_queued");
157157
READS_ISSUED_IN_COMMIT_PROC = metricsContext.getSummary("read_commit_proc_issued", DetailLevel.BASIC);
158158
WRITES_ISSUED_IN_COMMIT_PROC = metricsContext.getSummary("write_commit_proc_issued", DetailLevel.BASIC);
159+
EPHEMERAL_NODE_LIMIT_VIOLATION = metricsContext.getCounter("ephemeral_node_limit_violation");
159160

160161
/**
161162
* Time spent by a read request in the commit processor.
@@ -386,6 +387,7 @@ private ServerMetrics(MetricsProvider metricsProvider) {
386387
public final Counter COMMITS_QUEUED;
387388
public final Summary READS_ISSUED_IN_COMMIT_PROC;
388389
public final Summary WRITES_ISSUED_IN_COMMIT_PROC;
390+
public final Counter EPHEMERAL_NODE_LIMIT_VIOLATION;
389391

390392
/**
391393
* Time spent by a read request in the commit processor.

zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -99,6 +99,10 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
9999

100100
public static final String ENABLE_EAGER_ACL_CHECK = "zookeeper.enableEagerACLCheck";
101101
public static final String SKIP_ACL = "zookeeper.skipACL";
102+
public static final String EPHEMERAL_NODES_TOTAL_BYTE_LIMIT_KEY = "zookeeper.ephemeralNodes.total.byte.limit";
103+
public static final int DEFAULT_EPHEMERAL_NODES_TOTAL_BYTE_LIMIT = -1;
104+
private static int ephemeralCountLimit;
105+
private static int ephemeralNodesTotalByteLimit;
102106

103107
// When enabled, will check ACL constraints appertained to the requests first,
104108
// before sending the requests to the quorum.
@@ -139,6 +143,9 @@ public class ZooKeeperServer implements SessionExpirer, ServerStats.Provider {
139143
closeSessionTxnEnabled = Boolean.parseBoolean(
140144
System.getProperty(CLOSE_SESSION_TXN_ENABLED, "true"));
141145
LOG.info("{} = {}", CLOSE_SESSION_TXN_ENABLED, closeSessionTxnEnabled);
146+
147+
ephemeralNodesTotalByteLimit = Integer.getInteger(EPHEMERAL_NODES_TOTAL_BYTE_LIMIT_KEY, DEFAULT_EPHEMERAL_NODES_TOTAL_BYTE_LIMIT);
148+
LOG.info("{} = {}",EPHEMERAL_NODES_TOTAL_BYTE_LIMIT_KEY, ephemeralNodesTotalByteLimit);
142149
}
143150

144151
// @VisibleForTesting
@@ -162,6 +169,8 @@ public static void setCloseSessionTxnEnabled(boolean enabled) {
162169
ZooKeeperServer.closeSessionTxnEnabled);
163170
}
164171

172+
public static int getEphemeralNodesTotalByteLimit() {return ephemeralNodesTotalByteLimit;}
173+
165174
protected ZooKeeperServerBean jmxServerBean;
166175
protected DataTreeBean jmxDataTreeBean;
167176

Lines changed: 120 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,120 @@
1+
package org.apache.zookeeper.server.quorum;
2+
3+
import org.apache.jute.BinaryOutputArchive;
4+
import org.apache.zookeeper.CreateMode;
5+
import org.apache.zookeeper.KeeperException;
6+
import org.apache.zookeeper.ZooDefs;
7+
import org.apache.zookeeper.ZooKeeper;
8+
import org.junit.AfterClass;
9+
import org.junit.BeforeClass;
10+
import org.junit.Test;
11+
12+
import java.util.concurrent.ExecutorService;
13+
import java.util.concurrent.Executors;
14+
import java.util.concurrent.TimeUnit;
15+
16+
import static org.junit.Assert.assertEquals;
17+
18+
public class EphemeralNodeThrottlingMultithreadTest extends QuorumPeerTestBase {
19+
20+
public static final String EPHEMERAL_BYTE_LIMIT_KEY = "zookeeper.ephemeralNodes.total.byte.limit";
21+
static final String TEST_PATH = "/ephemeral-throttling-multithread-test";
22+
static final int DEFAULT_EPHEMERALNODES_TOTAL_BYTE_LIMIT = (int) (Math.pow(2d, 20d) * .5);
23+
static final int NUM_SERVERS = 5;
24+
25+
@BeforeClass
26+
public static void setUpClass() {
27+
System.setProperty(EPHEMERAL_BYTE_LIMIT_KEY, Integer.toString(DEFAULT_EPHEMERALNODES_TOTAL_BYTE_LIMIT));
28+
}
29+
30+
@AfterClass
31+
public static void tearDownClass() {
32+
System.clearProperty(EPHEMERAL_BYTE_LIMIT_KEY);
33+
}
34+
35+
// Tests multithreaded creates and deletes against the leader and a follower server
36+
@Test
37+
public void multithreadedRequestsTest() throws Exception {
38+
// 50% of 1mb jute max buffer
39+
int totalEphemeralNodesByteLimit = (int) (Math.pow(2d, 20d) * .5);
40+
System.setProperty("zookeeper.ephemeralNodes.total.byte.limit", Integer.toString(totalEphemeralNodesByteLimit));
41+
42+
servers = LaunchServers(NUM_SERVERS);
43+
ZooKeeper leaderServer = servers.zk[servers.findLeader()];
44+
ZooKeeper followerServer = servers.zk[servers.findAnyFollower()];
45+
46+
runMultithreadedRequests(leaderServer);
47+
runMultithreadedRequests(followerServer);
48+
49+
int leaderSessionEphemeralsByteSum = 0;
50+
for (String nodePath : leaderServer.getEphemerals()) {
51+
leaderSessionEphemeralsByteSum += BinaryOutputArchive.getSerializedStringByteSize(nodePath);
52+
}
53+
// TODO: What % delta do we want to allow here?
54+
assertEquals(totalEphemeralNodesByteLimit, leaderSessionEphemeralsByteSum, totalEphemeralNodesByteLimit/20d);
55+
56+
int followerSessionEphemeralsByteSum = 0;
57+
for (String nodePath : leaderServer.getEphemerals()) {
58+
followerSessionEphemeralsByteSum += BinaryOutputArchive.getSerializedStringByteSize(nodePath);
59+
}
60+
assertEquals(totalEphemeralNodesByteLimit, followerSessionEphemeralsByteSum, totalEphemeralNodesByteLimit/20d);
61+
62+
servers.shutDownAllServers();
63+
}
64+
65+
private void runMultithreadedRequests(ZooKeeper server) {
66+
int threadPoolCount = 8;
67+
int deleteRequestThreads = 2;
68+
int createRequestThreads = threadPoolCount - deleteRequestThreads;
69+
// Spin up threads to repeatedly send CREATE requests to server
70+
ExecutorService executor = Executors.newFixedThreadPool(threadPoolCount);
71+
for (int i = 0; i < createRequestThreads; i++) {
72+
final int threadID = i;
73+
executor.submit(() ->{
74+
long startTime = System.currentTimeMillis();
75+
while (System.currentTimeMillis() - startTime < 10000) {
76+
try {
77+
server.create(TEST_PATH +"_"+threadID+"_", new byte[512], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL);
78+
} catch (KeeperException.TotalEphemeralLimitExceeded expectedException) {
79+
// Ignore Ephemeral Count exceeded exception, as this is expected to occur
80+
} catch (Exception e) {
81+
LOG.warn("Thread encountered an exception but ignored it:\n" + e.getMessage());
82+
}
83+
}
84+
});
85+
}
86+
87+
// Spin up threads to repeatedly send DELETE requests to server
88+
// After a 1-second sleep, this should run concurrently with the create threads, but then end before create threads end
89+
// so that we still have time to hit the limit and can then assert that limit was upheld correctly
90+
for (int i = 0; i < deleteRequestThreads; i++) {
91+
executor.submit(() -> {
92+
long startTime = System.currentTimeMillis();
93+
try {
94+
// Brief sleep to reduce chance that ephemeral nodes not yet created
95+
Thread.sleep(1000);
96+
while (System.currentTimeMillis() - startTime < 4000) {
97+
for (String ephemeralNode : server.getEphemerals()) {
98+
server.delete(ephemeralNode, -1);
99+
}
100+
}
101+
} catch (KeeperException.TotalEphemeralLimitExceeded expectedException) {
102+
// Ignore Ephemeral Count exceeded exception, as this is expected to occur
103+
} catch (Exception e) {
104+
LOG.warn("Thread encountered an exception but ignored it:\n" + e.getMessage());
105+
}
106+
});
107+
}
108+
109+
executor.shutdown();
110+
try {
111+
if(!executor.awaitTermination(12000, TimeUnit.MILLISECONDS)) {
112+
LOG.warn("Threads did not finish in the given time!");
113+
executor.shutdownNow();
114+
}
115+
} catch (InterruptedException e) {
116+
LOG.error(e.getMessage());
117+
executor.shutdownNow();
118+
}
119+
}
120+
}

0 commit comments

Comments
 (0)