summaryrefslogtreecommitdiff
path: root/zookeeper-server/src/test/java/org/apache/zookeeper/ClientRequestTimeoutTest.java
blob: 8fcc371acbd3b0922aef9b3e06287ca6b99c6160 (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
/**
 * 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;

import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;

import java.io.IOException;

import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.client.HostProvider;
import org.apache.zookeeper.server.quorum.QuorumPeerTestBase;
import org.apache.zookeeper.test.ClientBase;
import org.apache.zookeeper.test.ClientBase.CountdownWatcher;
import org.junit.Assert;
import org.junit.Test;

public class ClientRequestTimeoutTest extends QuorumPeerTestBase {
    private static final int SERVER_COUNT = 3;
    private boolean dropPacket = false;
    private int dropPacketType = ZooDefs.OpCode.create;

    @Test(timeout = 120000)
    public void testClientRequestTimeout() throws Exception {
        int requestTimeOut = 15000;
        System.setProperty("zookeeper.request.timeout",
                Integer.toString(requestTimeOut));
        final int clientPorts[] = new int[SERVER_COUNT];
        StringBuilder sb = new StringBuilder();
        String server;

        for (int i = 0; i < SERVER_COUNT; i++) {
            clientPorts[i] = PortAssignment.unique();
            server = "server." + i + "=127.0.0.1:" + PortAssignment.unique()
                    + ":" + PortAssignment.unique() + ":participant;127.0.0.1:"
                    + clientPorts[i];
            sb.append(server + "\n");
        }
        String currentQuorumCfgSection = sb.toString();
        MainThread mt[] = new MainThread[SERVER_COUNT];

        for (int i = 0; i < SERVER_COUNT; i++) {
            mt[i] = new MainThread(i, clientPorts[i], currentQuorumCfgSection,
                    false);
            mt[i].start();
        }

        // ensure server started
        for (int i = 0; i < SERVER_COUNT; i++) {
            Assert.assertTrue("waiting for server " + i + " being up",
                    ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i],
                            CONNECTION_TIMEOUT));
        }

        CountdownWatcher watch1 = new CountdownWatcher();
        CustomZooKeeper zk = new CustomZooKeeper(getCxnString(clientPorts),
                ClientBase.CONNECTION_TIMEOUT, watch1);
        watch1.waitForConnected(ClientBase.CONNECTION_TIMEOUT);

        String data = "originalData";
        // lets see one successful operation
        zk.create("/clientHang1", data.getBytes(), Ids.OPEN_ACL_UNSAFE,
                CreateMode.PERSISTENT_SEQUENTIAL);

        // now make environment for client hang
        dropPacket = true;
        dropPacketType = ZooDefs.OpCode.create;

        // Test synchronous API
        try {
            zk.create("/clientHang2", data.getBytes(), Ids.OPEN_ACL_UNSAFE,
                    CreateMode.PERSISTENT);
            fail("KeeperException is expected.");
        } catch (KeeperException exception) {
            assertEquals(KeeperException.Code.REQUESTTIMEOUT.intValue(),
                    exception.code().intValue());
        }

        // do cleanup
        zk.close();
        for (int i = 0; i < SERVER_COUNT; i++) {
            mt[i].shutdown();
        }
    }

    /**
     * @return connection string in the form of
     *         127.0.0.1:port1,127.0.0.1:port2,127.0.0.1:port3
     */
    private String getCxnString(int[] clientPorts) {
        StringBuffer hostPortBuffer = new StringBuffer();
        for (int i = 0; i < clientPorts.length; i++) {
            hostPortBuffer.append("127.0.0.1:");
            hostPortBuffer.append(clientPorts[i]);
            if (i != (clientPorts.length - 1)) {
                hostPortBuffer.append(',');
            }
        }
        return hostPortBuffer.toString();
    }

    class CustomClientCnxn extends ClientCnxn {

        public CustomClientCnxn(String chrootPath, HostProvider hostProvider,
                int sessionTimeout, ZooKeeper zooKeeper,
                ClientWatchManager watcher, ClientCnxnSocket clientCnxnSocket,
                boolean canBeReadOnly) throws IOException {
            super(chrootPath, hostProvider, sessionTimeout, zooKeeper, watcher,
                    clientCnxnSocket, canBeReadOnly);
        }

        @Override
        public void finishPacket(Packet p) {
            if (dropPacket && p.requestHeader.getType() == dropPacketType) {
                // do nothing, just return, it is the same as packet is dropped
                // by the network
                return;
            }
            super.finishPacket(p);
        }
    }

    class CustomZooKeeper extends ZooKeeper {
        public CustomZooKeeper(String connectString, int sessionTimeout,
                Watcher watcher) throws IOException {
            super(connectString, sessionTimeout, watcher);
        }

        @Override
        protected ClientCnxn createConnection(String chrootPath,
                HostProvider hostProvider, int sessionTimeout,
                ZooKeeper zooKeeper, ClientWatchManager watcher,
                ClientCnxnSocket clientCnxnSocket, boolean canBeReadOnly)
                        throws IOException {
            return new CustomClientCnxn(chrootPath, hostProvider,
                    sessionTimeout, zooKeeper, watcher, clientCnxnSocket,
                    canBeReadOnly);
        }
    }
}