Jackie-Jiang commented on a change in pull request #8083: URL: https://github.com/apache/pinot/pull/8083#discussion_r796100819
########## File path: pinot-core/src/main/java/org/apache/pinot/core/transport/ServerChannels.java ########## @@ -139,22 +143,32 @@ private void attachSSLHandler(SocketChannel ch) { } } - synchronized void sendRequest(String rawTableName, AsyncQueryResponse asyncQueryResponse, - ServerRoutingInstance serverRoutingInstance, InstanceRequest instanceRequest) + private void sendRequestOrTimeOut(String rawTableName, AsyncQueryResponse asyncQueryResponse, + ServerRoutingInstance serverRoutingInstance, byte[] requestBytes, long timeoutMs) + throws Exception { + if (_channelLock.tryLock(timeoutMs, TimeUnit.MILLISECONDS)) { + try { + sendRequest(rawTableName, asyncQueryResponse, serverRoutingInstance, requestBytes); + } finally { + _channelLock.unlock(); + } + } Review comment: (major) Throw timeout exception if it fails to acquire the lock ########## File path: pinot-core/src/main/java/org/apache/pinot/core/transport/ServerChannels.java ########## @@ -139,22 +143,32 @@ private void attachSSLHandler(SocketChannel ch) { } } - synchronized void sendRequest(String rawTableName, AsyncQueryResponse asyncQueryResponse, - ServerRoutingInstance serverRoutingInstance, InstanceRequest instanceRequest) + private void sendRequestOrTimeOut(String rawTableName, AsyncQueryResponse asyncQueryResponse, Review comment: Suggest renaming it to `sendRequest`. Passing in a timeout implies that it can time out ########## File path: pinot-core/src/main/java/org/apache/pinot/core/transport/ServerChannels.java ########## @@ -94,9 +97,10 @@ public void shutDown() { @ThreadSafe private class ServerChannel { - final TSerializer _serializer = new TSerializer(new TCompactProtocol.Factory()); final ServerRoutingInstance _serverRoutingInstance; final Bootstrap _bootstrap; + // lock to protect channel as requests must be written into channel sequentially + private final ReentrantLock _channelLock = new ReentrantLock(); Review comment: (minor) `private` is redundant as the whole class is private ########## File path: pinot-core/src/main/java/org/apache/pinot/core/transport/ServerChannels.java ########## @@ -81,10 +83,11 @@ public ServerChannels(QueryRouter queryRouter, BrokerMetrics brokerMetrics, TlsC } public void sendRequest(String rawTableName, AsyncQueryResponse asyncQueryResponse, - ServerRoutingInstance serverRoutingInstance, InstanceRequest instanceRequest) + ServerRoutingInstance serverRoutingInstance, InstanceRequest instanceRequest, long timeoutMs) throws Exception { + byte[] requestBytes = _serializer.serialize(instanceRequest); Review comment: +1 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org