summaryrefslogtreecommitdiff
path: root/zookeeper-server/src/test/java/org/apache/zookeeper/server/RequestThrottlerTest.java
blob: 9eee9269c6eee99022e8c0a9ee2c6ef4a4879204 (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
/**
 * Licensed to the Apache Software Foundation (ASF) under one
 * or more contributor license agreements.  See the NOTICE file
 * distributed with this work for additional information
 * regarding copyright ownership.  The ASF licenses this file
 * to you under the Apache License, Version 2.0 (the
 * "License"); you may not use this file except in compliance
 * with the License.  You may obtain a copy of the License at
 *
 *     http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.zookeeper.server;

import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.PortAssignment;
import org.apache.zookeeper.ZKTestCase;
import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.common.Time;
import org.apache.zookeeper.metrics.MetricsUtils;
import org.apache.zookeeper.test.ClientBase;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.File;
import java.io.IOException;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;

import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.lessThan;

public class RequestThrottlerTest extends ZKTestCase {
    private static final Logger LOG = LoggerFactory.getLogger(RequestThrottlerTest.class);

    private static String HOSTPORT = "127.0.0.1:" + PortAssignment.unique();
    private final static int TOTAL_REQUESTS = 5;
    private final static int STALL_TIME = 5000;

    // latch to hold requests in the PrepRequestProcessor to
    // keep them from going down the pipeline to reach the final
    // request processor, where the number of in process requests
    // will be decreased
    CountDownLatch resumeProcess = null;

    // latch to make sure all requests are submitted
    CountDownLatch submitted = null;

    // latch to make sure all requests entered the pipeline
    CountDownLatch entered = null;

    ZooKeeperServer zks = null;
    ServerCnxnFactory f = null;
    ZooKeeper zk = null;

    @Before
    public void setup() throws Exception {
        // start a server and create a client
        File tmpDir = ClientBase.createTmpDir();
        ClientBase.setupTestEnv();
        zks = new TestZooKeeperServer(tmpDir, tmpDir, 3000);
        final int PORT = Integer.parseInt(HOSTPORT.split(":")[1]);
        f = ServerCnxnFactory.createFactory(PORT, -1);
        f.startup(zks);
        LOG.info("starting up the zookeeper server .. waiting");
        Assert.assertTrue("waiting for server being up",
                ClientBase.waitForServerUp(HOSTPORT, CONNECTION_TIMEOUT));

        resumeProcess = null;
        submitted = null;

        zk = ClientBase.createZKClient(HOSTPORT);
    }

    @After
    public void tearDown() throws Exception {
        // shut down the server and the client
        if (null != zk) {
            zk.close();
        }

        if (null != f) {
            f.shutdown();
        }
        if (null != zks) {
            zks.shutdown();
        }
    }

    // TestZooKeeperServer
    // 1. uses our version of PrepRequestProcessor, which can hold the request as long as we want
    // 2. count the number of submitted requests
    class TestZooKeeperServer extends ZooKeeperServer {
        public TestZooKeeperServer(File snapDir, File logDir, int tickTime) throws IOException {
            super(snapDir, logDir, tickTime);
        }

        @Override
        protected void setupRequestProcessors() {
            RequestProcessor finalProcessor = new FinalRequestProcessor(this);
            RequestProcessor syncProcessor = new SyncRequestProcessor(this,
                    finalProcessor);
            ((SyncRequestProcessor) syncProcessor).start();
            firstProcessor = new TestPrepRequestProcessor(this, syncProcessor);
            ((TestPrepRequestProcessor) firstProcessor).start();
        }

        @Override
        public void submitRequest(Request si) {
            if (null != submitted) {
                submitted.countDown();
            }
            super.submitRequest(si);
        }
    }

    class TestPrepRequestProcessor extends PrepRequestProcessor {
        public TestPrepRequestProcessor(ZooKeeperServer zks, RequestProcessor syncProcessor) {
            super(zks, syncProcessor);
        }

        @Override
        protected void pRequest(Request request) throws RequestProcessorException {
            // keep the request in the processor as long as we want
            if (resumeProcess != null) {
                try {
                    resumeProcess.await(20, TimeUnit.SECONDS);
                } catch (Exception e) {

                }
            }

            if (entered != null) {
                entered.countDown();
            }

            super.pRequest(request);
        }
    }

    @Test
    public void testRequestThrottler() throws Exception {
        ServerMetrics.getMetrics().resetAll();

        // we only allow two requests in the pipeline
        RequestThrottler.setMaxRequests(2);

        RequestThrottler.setStallTime(STALL_TIME);
        RequestThrottler.setDropStaleRequests(false);

        // no requests can go through the pipeline unless we raise the latch
        resumeProcess = new CountDownLatch(1);
        submitted = new CountDownLatch(TOTAL_REQUESTS);
        entered = new CountDownLatch(TOTAL_REQUESTS);

        // send 5 requests asynchronously
        for (int i =0; i < TOTAL_REQUESTS; i++) {
            zk.create("/request_throttle_test- " + i , ("/request_throttle_test- " + i).getBytes(),
                    ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT, (rc, path, ctx, name) -> {}, null);
        }

        // make sure the server received all 5 requests
        submitted.await(5, TimeUnit.SECONDS);
        Map<String, Object> metrics = MetricsUtils.currentServerMetrics();

        // but only two requests can get into the pipeline because of the throttler
        Assert.assertEquals(2L, (long)metrics.get("prep_processor_request_queued"));
        Assert.assertEquals(1L, (long)metrics.get("request_throttle_wait_count"));

        // let the requests go through the pipeline and the throttler will be waken up to allow more requests
        // to enter the pipeline
        resumeProcess.countDown();
        entered.await(STALL_TIME, TimeUnit.MILLISECONDS);

        metrics = MetricsUtils.currentServerMetrics();
        Assert.assertEquals(TOTAL_REQUESTS, (long)metrics.get("prep_processor_request_queued"));
    }

    @Test
    public void testDropStaleRequests() throws Exception {
        ServerMetrics.getMetrics().resetAll();

        // we only allow two requests in the pipeline
        RequestThrottler.setMaxRequests(2);

        RequestThrottler.setStallTime(STALL_TIME);

        RequestThrottler.setDropStaleRequests(true);

        // no requests can go through the pipeline unless we raise the latch
        resumeProcess = new CountDownLatch(1);
        submitted = new CountDownLatch(TOTAL_REQUESTS);

        // send 5 requests asynchronously
        for (int i=0; i<TOTAL_REQUESTS; i++) {
            zk.create("/request_throttle_test- " + i , ("/request_throttle_test- " + i).getBytes(),
                    ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT, (rc, path, ctx, name) -> {}, null);
        }

        // make sure the server received all 5 requests
        submitted.await(5, TimeUnit.SECONDS);
        Map<String, Object> metrics = MetricsUtils.currentServerMetrics();

        // but only two requests can get into the pipeline because of the throttler
        Assert.assertEquals(2L, (long)metrics.get("prep_processor_request_queued"));
        Assert.assertEquals(1L, (long)metrics.get("request_throttle_wait_count"));

        for (ServerCnxn cnxn : f.cnxns){
            cnxn.setStale();
        }
        zk = null;

        resumeProcess.countDown();
        LOG.info("raise the latch");

        while (zks.getInflight() > 0) {
            Thread.sleep(50);
        }

        // the rest of the 3 requests will be dropped
        // but only the first one for a connection will be counted
        metrics = MetricsUtils.currentServerMetrics();
        Assert.assertEquals(2L, (long)metrics.get("prep_processor_request_queued"));
        Assert.assertEquals(1, (long)metrics.get("stale_requests_dropped"));
    }
}