szetszwo commented on code in PR #740:
URL: https://github.com/apache/ratis/pull/740#discussion_r1281051523
##########
ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientStreamRpc.java:
##########
@@ -417,24 +379,48 @@ protected void decode(ChannelHandlerContext context,
ByteBuf buf, List<Object> o
public CompletableFuture<DataStreamReply> streamAsync(DataStreamRequest
request) {
final CompletableFuture<DataStreamReply> f = new CompletableFuture<>();
ClientInvocationId clientInvocationId =
ClientInvocationId.valueOf(request.getClientId(), request.getStreamId());
- final ReplyQueue q = replies.computeIfAbsent(clientInvocationId, key ->
new ReplyQueue());
- if (!q.offer(f)) {
- f.completeExceptionally(new IllegalStateException(this + ": Failed to
offer a future for " + request));
- return f;
- }
- final Channel channel = connection.getChannelUninterruptibly();
- if (channel == null) {
- f.completeExceptionally(new AlreadyClosedException(this + ": Failed to
send " + request));
- return f;
+
+ final NettyClientReply.ReplyQueue q =
replies.getReplyQueue(clientInvocationId);
+ ChannelFuture channelFuture;
+ final Channel channel;
+ LOG.debug("{}: write begin {}", this, request);
+ synchronized (q) {
+ channel = connection.getChannelUninterruptibly();
+ if (channel == null) {
+ f.completeExceptionally(new AlreadyClosedException(this + ": Failed to
send " + request));
+ return f;
+ }
+ q.offer(request, f);
+ final Function<DataStreamRequest, ChannelFuture> writeMethod =
outstandingRequests.write(request)?
+ channel::writeAndFlush: channel::write;
+ channelFuture = writeMethod.apply(request);
}
- LOG.debug("{}: write {}", this, request);
- final Function<DataStreamRequest, ChannelFuture> writeMethod =
outstandingRequests.write(request)?
- channel::writeAndFlush: channel::write;
- writeMethod.apply(request).addListener(future -> {
+ channelFuture.addListener(future -> {
if (!future.isSuccess()) {
- final IOException e = new IOException(this + ": Failed to send " +
request, future.cause());
+ q.poll(request);
+
+ Throwable cause = future.cause();
+ final IOException e = new IOException(this + ": Failed send message "
+ request + " to server " +
+ channel.remoteAddress(), cause);
LOG.error("Channel write failed", e);
f.completeExceptionally(e);
+ } else {
+ LOG.debug("{}: write after {}", this, request);
+
+ final boolean isClose =
StandardWriteOption.CLOSE.isOneOf(request.getWriteOptions());
Review Comment:
Let's use the new API.
```java
final boolean isClose =
request.getWriteOptionList().contains(StandardWriteOption.CLOSE);
```
##########
ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientReply.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.ratis.netty.client;
+
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.proto.RaftProtos;
+import org.apache.ratis.protocol.ClientInvocationId;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.protocol.DataStreamRequest;
+import org.apache.ratis.thirdparty.io.netty.util.concurrent.ScheduledFuture;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+public class NettyClientReply {
Review Comment:
Let's call it `NettyClientReplies`.
##########
ratis-netty/src/main/java/org/apache/ratis/netty/client/NettyClientReply.java:
##########
@@ -0,0 +1,196 @@
+/*
+ * 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.ratis.netty.client;
+
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.proto.RaftProtos;
+import org.apache.ratis.protocol.ClientInvocationId;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.protocol.DataStreamRequest;
+import org.apache.ratis.thirdparty.io.netty.util.concurrent.ScheduledFuture;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+public class NettyClientReply {
+
+ public static final Logger LOG =
LoggerFactory.getLogger(NettyClientReply.class);
+
+ private final ConcurrentMap<ClientInvocationId, ReplyQueue> replies = new
ConcurrentHashMap<>();
+
+ public ReplyQueue getReplyQueue(ClientInvocationId clientInvocationId) {
+ return replies.computeIfAbsent(clientInvocationId, key -> new
ReplyQueue());
+ }
+
+ public ReplyQueue getEmptyQueue() {
+ return new ReplyQueue();
+ }
+
+ class ReplyQueue {
Review Comment:
Let's call it `ReplyMap`.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]