[
https://issues.apache.org/jira/browse/ZOOKEEPER-2280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15943733#comment-15943733
]
ASF GitHub Bot commented on ZOOKEEPER-2280:
-------------------------------------------
Github user afine commented on a diff in the pull request:
https://github.com/apache/zookeeper/pull/208#discussion_r108230944
--- Diff:
src/java/test/org/apache/zookeeper/server/NettyServerCnxnTest.java ---
@@ -84,4 +98,78 @@ public void testSendCloseSession() throws Exception {
zk.close();
}
}
+
+ @Test(timeout = 30000)
+ public void testMaxClientConnectionsReached() throws Exception {
+ final int maxClientCnxns = 4;
+ final int numClients = 10;
+ createAndTestConnections(numClients, maxClientCnxns,
maxClientCnxns);
+ }
+
+ @Test(timeout = 30000)
+ public void testMaxClientConnectionsDisabled() throws Exception {
+ final int maxClientCnxns = 0; // disabled cnxns limit
+ final int numClients = 10;
+ createAndTestConnections(numClients, maxClientCnxns, numClients);
+ }
+
+ private void createAndTestConnections(int numClients, int
maxClientCnxns, int cnxnsAccepted) throws Exception {
+
+ File tmpDir = ClientBase.createTmpDir();
+ final int CLIENT_PORT = PortAssignment.unique();
+
+ ZooKeeperServer zks = new ZooKeeperServer(tmpDir, tmpDir, 3000);
+ ServerCnxnFactory scf =
ServerCnxnFactory.createFactory(CLIENT_PORT, maxClientCnxns);
+ scf.startup(zks);
+
+ try {
+ assertTrue("waiting for server being up",
+ ClientBase.waitForServerUp("127.0.0.1:" + CLIENT_PORT,
CONNECTION_TIMEOUT));
+ assertTrue("Didn't instantiate ServerCnxnFactory with
NettyServerCnxnFactory!",
+ scf instanceof NettyServerCnxnFactory);
+
+ assertEquals(0, scf.getNumAliveConnections());
+
+ assertTrue(cnxnsAccepted <= numClients);
+
+ final CountDownLatch countDownLatch = new
CountDownLatch(cnxnsAccepted);
+
+ TestableZooKeeper[] clients = new
TestableZooKeeper[numClients];
+ for (int i = 0; i < numClients; i++) {
+ clients[i] = new TestableZooKeeper("127.0.0.1:" +
CLIENT_PORT, 3000, new Watcher() {
+ @Override
+ public void process(WatchedEvent event)
+ {
+ if (event.getState() ==
Event.KeeperState.SyncConnected) {
+ countDownLatch.countDown();
+ }
+ }
+ });
+ }
+
+ countDownLatch.await();
+
+ assertEquals(cnxnsAccepted, scf.getNumAliveConnections());
+
+ ConcurrentMap<InetAddress, Set<NettyServerCnxn>> ipMap =
((NettyServerCnxnFactory) scf).ipMap;
+ assertEquals(1, ipMap.size());
+ Set<NettyServerCnxn> set =
ipMap.get(ipMap.keySet().toArray()[0]);
+ assertEquals(cnxnsAccepted, set.size());
+
+ int connected = 0;
+ for (int i = 0; i < numClients; i++) {
+ if (clients[i].getState().isConnected()) connected++;
--- End diff --
nit: I think we prefer multi-line if statements.
> NettyServerCnxnFactory doesn't honor maxClientCnxns param
> ---------------------------------------------------------
>
> Key: ZOOKEEPER-2280
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2280
> Project: ZooKeeper
> Issue Type: Bug
> Components: server
> Affects Versions: 3.4.6, 3.5.0, 3.5.1
> Reporter: Edward Ribeiro
> Assignee: Edward Ribeiro
> Fix For: 3.5.4, 3.6.0
>
> Attachments: ZOOKEEPER-2280.2.patch, ZOOKEEPER-2280.patch
>
>
> Even though NettyServerCnxnFactory has maxClientCnxns (default to 60) it
> doesn't enforce this limit in the code.
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)