Skip to content

Commit 4b686ed

Browse files
committed
ZOOKEEPER-4749: Respect zookeeper.request.timeout also for asynchronous api
Currently, `zookeeper.request.timeout` is only respected in synchronous api. I think there are should be no much differences between following two. ```java String createdPath = zk.create("/path", data, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); ``` ```java CompletableFuture<String> future = new CompletableFuture<>(); zk.create("/path", data, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT, (rc, path, ctx, name) -> { if (rc == 0) { future.complete(name); } else { future.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc), path)); } }, null); String createdPath = future.join(); ``` After this pr, we are able to unify synchronous api through calls to asynchronous api as [review comments][review-comments] pointed out if we feel there are too much identical code between synchronous and asynchronous api. [review-comments]: #2068 (comment)
1 parent 66202cb commit 4b686ed

File tree

6 files changed

+194
-192
lines changed

6 files changed

+194
-192
lines changed

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

Lines changed: 86 additions & 64 deletions
Original file line numberDiff line numberDiff line change
@@ -218,7 +218,7 @@ static class AuthData {
218218
* If any request's response in not received in configured requestTimeout
219219
* then it is assumed that the response packet is lost.
220220
*/
221-
private long requestTimeout;
221+
private final long requestTimeout;
222222

223223
ZKWatchManager getWatcherManager() {
224224
return watchManager;
@@ -286,6 +286,8 @@ static class Packet {
286286

287287
WatchDeregistration watchDeregistration;
288288

289+
long deadline = Long.MAX_VALUE;
290+
289291
/** Convenience ctor */
290292
Packet(
291293
RequestHeader requestHeader,
@@ -414,7 +416,12 @@ public ClientCnxn(
414416

415417
this.sendThread = new SendThread(clientCnxnSocket);
416418
this.eventThread = new EventThread();
417-
initRequestTimeout();
419+
this.requestTimeout = clientConfig.getRequestTimeout();
420+
LOG.info(
421+
"{} value is {}. feature enabled={}",
422+
ZKClientConfig.ZOOKEEPER_REQUEST_TIMEOUT,
423+
requestTimeout,
424+
requestTimeout > 0);
418425
}
419426

420427
public void start() {
@@ -728,8 +735,6 @@ protected void finishPacket(Packet p) {
728735
p.replyHeader.setErr(Code.OK.intValue());
729736
}
730737
}
731-
} catch (KeeperException.NoWatcherException nwe) {
732-
p.replyHeader.setErr(nwe.code().intValue());
733738
} catch (KeeperException ke) {
734739
p.replyHeader.setErr(ke.code().intValue());
735740
}
@@ -765,21 +770,26 @@ protected void onConnecting(InetSocketAddress addr) {
765770

766771
}
767772

768-
private void conLossPacket(Packet p) {
773+
private Code abortPacket(Packet p, Code cause) {
769774
if (p.replyHeader == null) {
770-
return;
775+
return cause;
771776
}
772777
switch (state) {
773-
case AUTH_FAILED:
774-
p.replyHeader.setErr(KeeperException.Code.AUTHFAILED.intValue());
775-
break;
776-
case CLOSED:
777-
p.replyHeader.setErr(KeeperException.Code.SESSIONEXPIRED.intValue());
778-
break;
779-
default:
780-
p.replyHeader.setErr(KeeperException.Code.CONNECTIONLOSS.intValue());
778+
case AUTH_FAILED:
779+
p.replyHeader.setErr(KeeperException.Code.AUTHFAILED.intValue());
780+
break;
781+
case CLOSED:
782+
p.replyHeader.setErr(KeeperException.Code.SESSIONEXPIRED.intValue());
783+
break;
784+
default:
785+
p.replyHeader.setErr(cause.intValue());
781786
}
782787
finishPacket(p);
788+
return Code.CONNECTIONLOSS;
789+
}
790+
791+
private void conLossPacket(Packet p) {
792+
abortPacket(p, Code.CONNECTIONLOSS);
783793
}
784794

785795
private volatile long lastZxid;
@@ -852,6 +862,8 @@ void readResponse(ByteBuffer incomingBuffer) throws IOException {
852862

853863
replyHdr.deserialize(bbia, "header");
854864
switch (replyHdr.getXid()) {
865+
case SET_WATCHES_XID:
866+
return;
855867
case PING_XID:
856868
LOG.debug("Got ping response for session id: 0x{} after {}ms.",
857869
Long.toHexString(sessionId),
@@ -1116,6 +1128,39 @@ private void logStartConnect(InetSocketAddress addr) {
11161128
}
11171129
}
11181130

1131+
private long requestDeadline() {
1132+
if (requestTimeout == 0) {
1133+
return Long.MAX_VALUE;
1134+
}
1135+
1136+
// The correctness of following code depends on several implementation details:
1137+
// 1. Polling of outgoingQueue happens only in SendThread.
1138+
// 2. Adding to pendingQueue happens only in SendThread.
1139+
//
1140+
// It is possible for netty socket to readResponse for first pendingQueue entry
1141+
// while we are checking deadline for the same entry. So, it is possible that
1142+
// a request was responded near deadline, but we disconnect the session. Given
1143+
// that we are dealing with timeout, this should not be much matter.
1144+
//
1145+
// In long term, we should sequence all pendingQueue operations to SendThread.
1146+
1147+
Packet p;
1148+
synchronized (pendingQueue) {
1149+
p = pendingQueue.peek();
1150+
}
1151+
if (p != null) {
1152+
return p.deadline;
1153+
}
1154+
1155+
for (Packet packet : outgoingQueue) {
1156+
if (packet.requestHeader != null && packet.requestHeader.getXid() >= 0) {
1157+
return packet.deadline;
1158+
}
1159+
}
1160+
1161+
return Long.MAX_VALUE;
1162+
}
1163+
11191164
@Override
11201165
@SuppressFBWarnings("JLM_JSR166_UTILCONCURRENT_MONITORENTER")
11211166
public void run() {
@@ -1192,6 +1237,14 @@ public void run() {
11921237
LOG.warn(warnInfo);
11931238
throw new SessionTimeoutException(warnInfo);
11941239
}
1240+
long deadline = requestDeadline();
1241+
if (deadline != Long.MAX_VALUE) {
1242+
long now = Time.currentElapsedTime();
1243+
if (now >= deadline) {
1244+
throw new KeeperException.RequestTimeoutException();
1245+
}
1246+
to = Integer.min(to, (int) (deadline - now));
1247+
}
11951248
if (state.isConnected()) {
11961249
//1000(1 second) is to prevent race condition missing to send the second ping
11971250
//also make sure not to send too many pings when readTimeout is small
@@ -1240,9 +1293,14 @@ public void run() {
12401293
serverAddress,
12411294
e);
12421295

1296+
Code cause = Code.CONNECTIONLOSS;
1297+
if (e instanceof KeeperException) {
1298+
cause = ((KeeperException) e).code();
1299+
}
1300+
12431301
// At this point, there might still be new packets appended to outgoingQueue.
12441302
// they will be handled in next connection or cleared up if closed.
1245-
cleanAndNotifyState();
1303+
cleanAndNotifyState(cause);
12461304
}
12471305
}
12481306
}
@@ -1268,8 +1326,8 @@ public void run() {
12681326
"SendThread exited loop for session: 0x" + Long.toHexString(getSessionId()));
12691327
}
12701328

1271-
private void cleanAndNotifyState() {
1272-
cleanup();
1329+
private void cleanAndNotifyState(Code cause) {
1330+
cleanup(cause);
12731331
if (state.isAlive()) {
12741332
eventThread.queueEvent(new WatchedEvent(Event.EventType.None, Event.KeeperState.Disconnected, null));
12751333
}
@@ -1328,10 +1386,14 @@ private void pingRwServer() throws RWServerFoundException {
13281386
}
13291387

13301388
private void cleanup() {
1389+
cleanup(Code.CONNECTIONLOSS);
1390+
}
1391+
1392+
private void cleanup(Code cause) {
13311393
clientCnxnSocket.cleanup();
13321394
synchronized (pendingQueue) {
13331395
for (Packet p : pendingQueue) {
1334-
conLossPacket(p);
1396+
cause = abortPacket(p, cause);
13351397
}
13361398
pendingQueue.clear();
13371399
}
@@ -1341,7 +1403,7 @@ private void cleanup() {
13411403
Iterator<Packet> iter = outgoingQueue.iterator();
13421404
while (iter.hasNext()) {
13431405
Packet p = iter.next();
1344-
conLossPacket(p);
1406+
cause = abortPacket(p, cause);
13451407
iter.remove();
13461408
}
13471409
}
@@ -1525,37 +1587,13 @@ public ReplyHeader submitRequest(
15251587
watchRegistration,
15261588
watchDeregistration);
15271589
synchronized (packet) {
1528-
if (requestTimeout > 0) {
1529-
// Wait for request completion with timeout
1530-
waitForPacketFinish(r, packet);
1531-
} else {
1532-
// Wait for request completion infinitely
1533-
while (!packet.finished) {
1534-
packet.wait();
1535-
}
1590+
while (!packet.finished) {
1591+
packet.wait();
15361592
}
15371593
}
1538-
if (r.getErr() == Code.REQUESTTIMEOUT.intValue()) {
1539-
sendThread.cleanAndNotifyState();
1540-
}
15411594
return r;
15421595
}
15431596

1544-
/**
1545-
* Wait for request completion with timeout.
1546-
*/
1547-
private void waitForPacketFinish(ReplyHeader r, Packet packet) throws InterruptedException {
1548-
long waitStartTime = Time.currentElapsedTime();
1549-
while (!packet.finished) {
1550-
packet.wait(requestTimeout);
1551-
if (!packet.finished && ((Time.currentElapsedTime() - waitStartTime) >= requestTimeout)) {
1552-
LOG.error("Timeout error occurred for the packet '{}'.", packet);
1553-
r.setErr(Code.REQUESTTIMEOUT.intValue());
1554-
break;
1555-
}
1556-
}
1557-
}
1558-
15591597
public void saslCompleted() {
15601598
sendThread.getClientCnxnSocket().saslCompleted();
15611599
}
@@ -1612,6 +1650,9 @@ public Packet queuePacket(
16121650
packet.clientPath = clientPath;
16131651
packet.serverPath = serverPath;
16141652
packet.watchDeregistration = watchDeregistration;
1653+
if (requestTimeout != 0 && h.getXid() >= 0) {
1654+
packet.deadline = Time.currentElapsedTime() + requestTimeout;
1655+
}
16151656
// The synchronized block here is for two purpose:
16161657
// 1. synchronize with the final cleanup() in SendThread.run() to avoid race
16171658
// 2. synchronized against each packet. So if a closeSession packet is added,
@@ -1669,25 +1710,6 @@ public LocalCallback(AsyncCallback cb, int rc, String path, Object ctx) {
16691710

16701711
}
16711712

1672-
private void initRequestTimeout() {
1673-
try {
1674-
requestTimeout = clientConfig.getLong(
1675-
ZKClientConfig.ZOOKEEPER_REQUEST_TIMEOUT,
1676-
ZKClientConfig.ZOOKEEPER_REQUEST_TIMEOUT_DEFAULT);
1677-
LOG.info(
1678-
"{} value is {}. feature enabled={}",
1679-
ZKClientConfig.ZOOKEEPER_REQUEST_TIMEOUT,
1680-
requestTimeout,
1681-
requestTimeout > 0);
1682-
} catch (NumberFormatException e) {
1683-
LOG.error(
1684-
"Configured value {} for property {} can not be parsed to long.",
1685-
clientConfig.getProperty(ZKClientConfig.ZOOKEEPER_REQUEST_TIMEOUT),
1686-
ZKClientConfig.ZOOKEEPER_REQUEST_TIMEOUT);
1687-
throw e;
1688-
}
1689-
}
1690-
16911713
public ZooKeeperSaslClient getZooKeeperSaslClient() {
16921714
return sendThread.getZooKeeperSaslClient();
16931715
}

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

Lines changed: 2 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,6 @@
3434
import java.util.concurrent.LinkedBlockingDeque;
3535
import org.apache.zookeeper.ClientCnxn.EndOfStreamException;
3636
import org.apache.zookeeper.ClientCnxn.Packet;
37-
import org.apache.zookeeper.ZooDefs.OpCode;
3837
import org.apache.zookeeper.client.ZKClientConfig;
3938
import org.slf4j.Logger;
4039
import org.slf4j.LoggerFactory;
@@ -109,9 +108,7 @@ void doIO(Queue<Packet> pendingQueue, ClientCnxn cnxn) throws InterruptedExcepti
109108
updateLastSend();
110109
// If we already started writing p, p.bb will already exist
111110
if (p.bb == null) {
112-
if ((p.requestHeader != null)
113-
&& (p.requestHeader.getType() != OpCode.ping)
114-
&& (p.requestHeader.getType() != OpCode.auth)) {
111+
if (p.requestHeader != null && p.requestHeader.getXid() >= 0) {
115112
p.requestHeader.setXid(cnxn.getXid());
116113
}
117114
p.createBB();
@@ -120,9 +117,7 @@ void doIO(Queue<Packet> pendingQueue, ClientCnxn cnxn) throws InterruptedExcepti
120117
if (!p.bb.hasRemaining()) {
121118
sentCount.getAndIncrement();
122119
outgoingQueue.removeFirstOccurrence(p);
123-
if (p.requestHeader != null
124-
&& p.requestHeader.getType() != OpCode.ping
125-
&& p.requestHeader.getType() != OpCode.auth) {
120+
if (p.requestHeader != null && p.requestHeader.getXid() >= 0) {
126121
synchronized (pendingQueue) {
127122
pendingQueue.add(p);
128123
}

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

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -354,9 +354,7 @@ private void doWrite(Queue<Packet> pendingQueue, Packet p, ClientCnxn cnxn) thro
354354
boolean anyPacketsSent = false;
355355
while (true) {
356356
if (p != WakeupPacket.getInstance()) {
357-
if ((p.requestHeader != null)
358-
&& (p.requestHeader.getType() != ZooDefs.OpCode.ping)
359-
&& (p.requestHeader.getType() != ZooDefs.OpCode.auth)) {
357+
if (p.requestHeader != null && p.requestHeader.getXid() >= 0) {
360358
p.requestHeader.setXid(cnxn.getXid());
361359
synchronized (pendingQueue) {
362360
pendingQueue.add(p);

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

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -514,6 +514,8 @@ static String getCodeMessage(Code code) {
514514
return "Quota has exceeded";
515515
case THROTTLEDOP:
516516
return "Op throttled due to high load";
517+
case REQUESTTIMEOUT:
518+
return "Request timeout";
517519
default:
518520
return "Unknown error " + code;
519521
}

zookeeper-server/src/main/java/org/apache/zookeeper/client/ZKClientConfig.java

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,13 +23,16 @@
2323
import org.apache.zookeeper.ZooKeeper;
2424
import org.apache.zookeeper.common.ZKConfig;
2525
import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
26+
import org.slf4j.Logger;
27+
import org.slf4j.LoggerFactory;
2628

2729
/**
2830
* Handles client specific properties
2931
* @since 3.5.2
3032
*/
3133
@InterfaceAudience.Public
3234
public class ZKClientConfig extends ZKConfig {
35+
private static final Logger LOG = LoggerFactory.getLogger(ZKClientConfig.class);
3336

3437
public static final String ZK_SASL_CLIENT_USERNAME = "zookeeper.sasl.client.username";
3538
public static final String ZK_SASL_CLIENT_USERNAME_DEFAULT = "zookeeper";
@@ -142,4 +145,18 @@ public long getLong(String key, long defaultValue) {
142145
return defaultValue;
143146
}
144147

148+
@InterfaceAudience.Private
149+
public long getRequestTimeout() {
150+
try {
151+
return getLong(
152+
ZKClientConfig.ZOOKEEPER_REQUEST_TIMEOUT,
153+
ZKClientConfig.ZOOKEEPER_REQUEST_TIMEOUT_DEFAULT);
154+
} catch (NumberFormatException e) {
155+
LOG.error(
156+
"Configured value {} for property {} can not be parsed to long.",
157+
getProperty(ZKClientConfig.ZOOKEEPER_REQUEST_TIMEOUT),
158+
ZKClientConfig.ZOOKEEPER_REQUEST_TIMEOUT);
159+
throw e;
160+
}
161+
}
145162
}

0 commit comments

Comments
 (0)