You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@uniffle.apache.org by "advancedxy (via GitHub)" <gi...@apache.org> on 2023/03/28 11:33:21 UTC

[GitHub] [incubator-uniffle] advancedxy commented on a diff in pull request #771: [#584] feat(netty): Add transport client pool for netty

advancedxy commented on code in PR #771:
URL: https://github.com/apache/incubator-uniffle/pull/771#discussion_r1150421787


##########
common/src/main/java/org/apache/uniffle/common/config/RssClientConf.java:
##########
@@ -43,4 +44,55 @@ public class RssClientConf {
       .defaultValue(ShuffleDataDistributionType.NORMAL)
       .withDescription("The type of partition shuffle data distribution, including normal and local_order. "
           + "The default value is normal. This config is only valid in Spark3.x");
+
+  public static final ConfigOption<Integer> NETTY_IO_CONNECT_TIMEOUT_MS = ConfigOptions
+      .key("rss.client.netty.io.connect.timeout.ms")
+      .intType()
+      .defaultValue(10 * 1000)
+      .withDescription("netty connect to server time out mills");
+
+  public static final ConfigOption<IOMode> NETTY_IO_MODE = ConfigOptions
+      .key("rss.client.netty.io.mode")
+      .enumType(IOMode.class)
+      .defaultValue(IOMode.NIO)
+      .withDescription("Netty EventLoopGroup backend, available options: NIO, EPOLL.");
+
+  public static final ConfigOption<Integer> NETTY_IO_CONNECTION_TIMEOUT_MS = ConfigOptions
+      .key("rss.client.netty.client.connection.timeout.ms")
+      .intType()
+      .defaultValue(10 * 60 * 1000)
+      .withDescription("connection active timeout");
+
+  public static final ConfigOption<Integer> NETTY_CLIENT_THREADS = ConfigOptions
+      .key("rss.client.netty.client.threads")
+      .intType()
+      .defaultValue(0)
+      .withDescription("Number of threads used in the client thread pool.");
+
+  public static final ConfigOption<Boolean> NETWORK_CLIENT_PREFER_DIRECT_BUFS = ConfigOptions
+      .key("rss.client.netty.client.prefer.direct.bufs")
+      .booleanType()
+      .defaultValue(true)
+      .withDescription("If true, we will prefer allocating off-heap byte buffers within Netty.");
+
+  public static final ConfigOption<Integer> NETTY_CLIENT_NUM_CONNECTIONS_PER_PEER = ConfigOptions
+      .key("rss.client.netty.client.connections.per.peer")
+      .intType()
+      .defaultValue(2)

Review Comment:
   is this too small?



##########
common/src/main/java/org/apache/uniffle/common/netty/client/TransportClient.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.uniffle.common.netty.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.SocketAddress;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.common.base.Preconditions;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.util.concurrent.Future;
+import io.netty.util.concurrent.GenericFutureListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.uniffle.common.netty.handle.TransportResponseHandler;
+import org.apache.uniffle.common.netty.protocol.Message;
+import org.apache.uniffle.common.util.NettyUtils;
+
+
+public class TransportClient implements Closeable {
+  private static final Logger logger = LoggerFactory.getLogger(TransportClient.class);
+
+  private Channel channel;
+  private TransportResponseHandler handler;
+  private volatile boolean timedOut;
+
+  private static final AtomicLong counter = new AtomicLong();
+
+  public TransportClient(Channel channel, TransportResponseHandler handler) {
+    this.channel = Preconditions.checkNotNull(channel);
+    this.handler = Preconditions.checkNotNull(handler);

Review Comment:
   Use `java.util.Objects#requireNonNull` instead?



##########
common/src/main/java/org/apache/uniffle/common/config/RssClientConf.java:
##########
@@ -43,4 +44,55 @@ public class RssClientConf {
       .defaultValue(ShuffleDataDistributionType.NORMAL)
       .withDescription("The type of partition shuffle data distribution, including normal and local_order. "
           + "The default value is normal. This config is only valid in Spark3.x");
+
+  public static final ConfigOption<Integer> NETTY_IO_CONNECT_TIMEOUT_MS = ConfigOptions
+      .key("rss.client.netty.io.connect.timeout.ms")
+      .intType()
+      .defaultValue(10 * 1000)
+      .withDescription("netty connect to server time out mills");
+
+  public static final ConfigOption<IOMode> NETTY_IO_MODE = ConfigOptions
+      .key("rss.client.netty.io.mode")

Review Comment:
   @leixm added `rss.server.netty.epoll.enable` for netty sever,do you guys it's better to rename that configuration to `rss.server.netty.io.mode` instead? So they are more consistent. 



##########
common/src/main/java/org/apache/uniffle/common/netty/client/TransportClientFactory.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.uniffle.common.netty.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.base.Preconditions;
+import io.netty.bootstrap.Bootstrap;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.uniffle.common.netty.IOMode;
+import org.apache.uniffle.common.netty.TransportFrameDecoder;
+import org.apache.uniffle.common.netty.handle.TransportResponseHandler;
+import org.apache.uniffle.common.util.JavaUtils;
+import org.apache.uniffle.common.util.NettyUtils;
+
+public class TransportClientFactory implements Closeable {
+
+  /**
+   * A simple data structure to track the pool of clients between two peer nodes.
+   */
+  private static class ClientPool {
+    TransportClient[] clients;
+    Object[] locks;
+
+    ClientPool(int size) {
+      clients = new TransportClient[size];
+      locks = new Object[size];
+      for (int i = 0; i < size; i++) {
+        locks[i] = new Object();
+      }
+    }
+  }
+
+  private static final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class);
+
+  private final TransportContext context;
+  private final TransportConf conf;
+  private final ConcurrentHashMap<SocketAddress, ClientPool> connectionPool;
+
+  /**
+   * Random number generator for picking connections between peers.
+   */
+  private final Random rand;
+
+  private final int numConnectionsPerPeer;
+
+  private final Class<? extends Channel> socketChannelClass;
+  private EventLoopGroup workerGroup;
+  private PooledByteBufAllocator pooledAllocator;
+
+  public TransportClientFactory(TransportContext context) {
+    this.context = Preconditions.checkNotNull(context);
+    this.conf = context.getConf();
+    this.connectionPool = new ConcurrentHashMap<>();
+    this.numConnectionsPerPeer = conf.numConnectionsPerPeer();
+    this.rand = new Random();
+
+    IOMode ioMode = conf.ioMode();
+    this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode);
+    this.workerGroup =
+        NettyUtils.createEventLoop(ioMode, conf.clientThreads(), "netty-rpc-client");
+    this.pooledAllocator =
+        NettyUtils.createPooledByteBufAllocator(
+            conf.preferDirectBufs(), false, conf.clientThreads());
+  }
+
+  public TransportClient createClient(String remoteHost, int remotePort, int partitionId)
+      throws IOException, InterruptedException {
+    return createClient(remoteHost, remotePort, partitionId, new TransportFrameDecoder());
+  }
+
+  public TransportClient createClient(
+      String remoteHost, int remotePort, int partitionId, ChannelInboundHandlerAdapter decoder)
+      throws IOException, InterruptedException {
+    // Get connection from the connection pool first.
+    // If it is not found or not active, create a new one.
+    // Use unresolved address here to avoid DNS resolution each time we creates a client.
+    final InetSocketAddress unresolvedAddress =
+        InetSocketAddress.createUnresolved(remoteHost, remotePort);
+
+    // Create the ClientPool if we don't have it yet.
+    ClientPool clientPool = connectionPool.get(unresolvedAddress);
+    if (clientPool == null) {
+      connectionPool.putIfAbsent(unresolvedAddress, new ClientPool(numConnectionsPerPeer));
+      clientPool = connectionPool.get(unresolvedAddress);
+    }

Review Comment:
   ```suggestion
       ClientPool clientPool = connectionPool.computeIfAbsent(unresolvedAddress, x -> new ClientPool(numConnectionsPerPeer));
   ```
   ?



##########
common/src/main/java/org/apache/uniffle/common/netty/client/TransportClientFactory.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.uniffle.common.netty.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.base.Preconditions;
+import io.netty.bootstrap.Bootstrap;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.uniffle.common.netty.IOMode;
+import org.apache.uniffle.common.netty.TransportFrameDecoder;
+import org.apache.uniffle.common.netty.handle.TransportResponseHandler;
+import org.apache.uniffle.common.util.JavaUtils;
+import org.apache.uniffle.common.util.NettyUtils;
+
+public class TransportClientFactory implements Closeable {
+
+  /**
+   * A simple data structure to track the pool of clients between two peer nodes.
+   */
+  private static class ClientPool {
+    TransportClient[] clients;
+    Object[] locks;
+
+    ClientPool(int size) {
+      clients = new TransportClient[size];
+      locks = new Object[size];
+      for (int i = 0; i < size; i++) {
+        locks[i] = new Object();
+      }
+    }
+  }
+
+  private static final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class);
+
+  private final TransportContext context;
+  private final TransportConf conf;
+  private final ConcurrentHashMap<SocketAddress, ClientPool> connectionPool;
+
+  /**
+   * Random number generator for picking connections between peers.
+   */
+  private final Random rand;
+
+  private final int numConnectionsPerPeer;
+
+  private final Class<? extends Channel> socketChannelClass;
+  private EventLoopGroup workerGroup;
+  private PooledByteBufAllocator pooledAllocator;
+
+  public TransportClientFactory(TransportContext context) {
+    this.context = Preconditions.checkNotNull(context);
+    this.conf = context.getConf();
+    this.connectionPool = new ConcurrentHashMap<>();
+    this.numConnectionsPerPeer = conf.numConnectionsPerPeer();
+    this.rand = new Random();
+
+    IOMode ioMode = conf.ioMode();
+    this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode);
+    this.workerGroup =
+        NettyUtils.createEventLoop(ioMode, conf.clientThreads(), "netty-rpc-client");
+    this.pooledAllocator =
+        NettyUtils.createPooledByteBufAllocator(
+            conf.preferDirectBufs(), false, conf.clientThreads());
+  }
+
+  public TransportClient createClient(String remoteHost, int remotePort, int partitionId)
+      throws IOException, InterruptedException {
+    return createClient(remoteHost, remotePort, partitionId, new TransportFrameDecoder());
+  }
+
+  public TransportClient createClient(
+      String remoteHost, int remotePort, int partitionId, ChannelInboundHandlerAdapter decoder)
+      throws IOException, InterruptedException {
+    // Get connection from the connection pool first.
+    // If it is not found or not active, create a new one.
+    // Use unresolved address here to avoid DNS resolution each time we creates a client.
+    final InetSocketAddress unresolvedAddress =
+        InetSocketAddress.createUnresolved(remoteHost, remotePort);
+
+    // Create the ClientPool if we don't have it yet.
+    ClientPool clientPool = connectionPool.get(unresolvedAddress);
+    if (clientPool == null) {
+      connectionPool.putIfAbsent(unresolvedAddress, new ClientPool(numConnectionsPerPeer));
+      clientPool = connectionPool.get(unresolvedAddress);
+    }
+
+    int clientIndex =
+        partitionId < 0 ? rand.nextInt(numConnectionsPerPeer) : partitionId % numConnectionsPerPeer;
+    TransportClient cachedClient = clientPool.clients[clientIndex];
+
+    if (cachedClient != null && cachedClient.isActive()) {
+      // Make sure that the channel will not timeout by updating the last use time of the
+      // handler. Then check that the client is still alive, in case it timed out before
+      // this code was able to update things.
+      TransportResponseHandler handler =
+          cachedClient.getChannel().pipeline().get(TransportResponseHandler.class);
+
+      if (cachedClient.isActive()) {
+        logger.trace(
+            "Returning cached connection to {}: {}", cachedClient.getSocketAddress(), cachedClient);
+        return cachedClient;
+      }
+    }
+
+    // If we reach here, we don't have an existing connection open. Let's create a new one.
+    // Multiple threads might race here to create new connections. Keep only one of them active.
+    final long preResolveHost = System.nanoTime();
+    final InetSocketAddress resolvedAddress = new InetSocketAddress(remoteHost, remotePort);
+    final long hostResolveTimeMs = (System.nanoTime() - preResolveHost) / 1000000;
+    if (hostResolveTimeMs > 2000) {
+      logger.warn("DNS resolution for {} took {} ms", resolvedAddress, hostResolveTimeMs);
+    } else {
+      logger.trace("DNS resolution for {} took {} ms", resolvedAddress, hostResolveTimeMs);
+    }
+
+    synchronized (clientPool.locks[clientIndex]) {
+      cachedClient = clientPool.clients[clientIndex];
+
+      if (cachedClient != null) {
+        if (cachedClient.isActive()) {
+          logger.trace("Returning cached connection to {}: {}", resolvedAddress, cachedClient);
+          return cachedClient;
+        } else {
+          logger.info("Found inactive connection to {}, creating a new one.", resolvedAddress);
+        }
+      }
+      clientPool.clients[clientIndex] = internalCreateClient(resolvedAddress, decoder);

Review Comment:
   Could you wrap these block of code into the ClientPool structure?
   
   Such as `clientPoo.createClientIfAbsent`, which should be similar as ConcurrentMap's computeIfAbsent?



##########
common/src/main/java/org/apache/uniffle/common/netty/client/TransportClientFactory.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.uniffle.common.netty.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.base.Preconditions;
+import io.netty.bootstrap.Bootstrap;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.uniffle.common.netty.IOMode;
+import org.apache.uniffle.common.netty.TransportFrameDecoder;
+import org.apache.uniffle.common.netty.handle.TransportResponseHandler;
+import org.apache.uniffle.common.util.JavaUtils;
+import org.apache.uniffle.common.util.NettyUtils;
+
+public class TransportClientFactory implements Closeable {
+
+  /**
+   * A simple data structure to track the pool of clients between two peer nodes.
+   */
+  private static class ClientPool {
+    TransportClient[] clients;
+    Object[] locks;
+
+    ClientPool(int size) {
+      clients = new TransportClient[size];
+      locks = new Object[size];
+      for (int i = 0; i < size; i++) {
+        locks[i] = new Object();
+      }
+    }
+  }
+
+  private static final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class);
+
+  private final TransportContext context;
+  private final TransportConf conf;
+  private final ConcurrentHashMap<SocketAddress, ClientPool> connectionPool;
+
+  /**
+   * Random number generator for picking connections between peers.
+   */
+  private final Random rand;
+
+  private final int numConnectionsPerPeer;
+
+  private final Class<? extends Channel> socketChannelClass;
+  private EventLoopGroup workerGroup;
+  private PooledByteBufAllocator pooledAllocator;
+
+  public TransportClientFactory(TransportContext context) {
+    this.context = Preconditions.checkNotNull(context);
+    this.conf = context.getConf();
+    this.connectionPool = new ConcurrentHashMap<>();

Review Comment:
   Use JavaUtils.newConcurrentMap?



##########
common/src/main/java/org/apache/uniffle/common/netty/client/TransportClient.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.uniffle.common.netty.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.SocketAddress;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.common.base.Preconditions;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.util.concurrent.Future;
+import io.netty.util.concurrent.GenericFutureListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.uniffle.common.netty.handle.TransportResponseHandler;
+import org.apache.uniffle.common.netty.protocol.Message;
+import org.apache.uniffle.common.util.NettyUtils;
+
+
+public class TransportClient implements Closeable {
+  private static final Logger logger = LoggerFactory.getLogger(TransportClient.class);
+
+  private Channel channel;
+  private TransportResponseHandler handler;
+  private volatile boolean timedOut;
+
+  private static final AtomicLong counter = new AtomicLong();
+
+  public TransportClient(Channel channel, TransportResponseHandler handler) {
+    this.channel = Preconditions.checkNotNull(channel);
+    this.handler = Preconditions.checkNotNull(handler);
+    this.timedOut = false;
+  }
+
+  public Channel getChannel() {
+    return channel;
+  }
+
+  public boolean isActive() {
+    return !timedOut && (channel.isOpen() || channel.isActive());
+  }
+
+  public SocketAddress getSocketAddress() {
+    return channel.remoteAddress();
+  }
+
+  public ChannelFuture sendShuffleData(Message message, RpcResponseCallback callback) {
+    if (logger.isTraceEnabled()) {
+      logger.trace("Pushing data to {}", NettyUtils.getRemoteAddress(channel));
+    }
+    long requestId = requestId();
+    handler.addResponseCallback(requestId, callback);
+    RpcChannelListener listener = new RpcChannelListener(requestId, callback);
+    return channel.writeAndFlush(message).addListener(listener);
+  }
+
+  public static long requestId() {
+    return counter.getAndIncrement();
+  }
+
+  public class StdChannelListener implements GenericFutureListener<Future<? super Void>> {

Review Comment:
   Seems like that it's better to declare it as `public static class`?



##########
common/src/main/java/org/apache/uniffle/common/netty/client/TransportContext.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.uniffle.common.netty.client;
+
+import io.netty.channel.ChannelInboundHandlerAdapter;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.handler.timeout.IdleStateHandler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.uniffle.common.netty.MessageEncoder;
+import org.apache.uniffle.common.netty.handle.TransportResponseHandler;
+
+public class TransportContext {
+  private static final Logger logger = LoggerFactory.getLogger(TransportContext.class);
+
+  private TransportConf transportConf;
+
+  private static final MessageEncoder ENCODER = MessageEncoder.INSTANCE;
+
+  public TransportContext(TransportConf transportConf) {
+    this.transportConf = transportConf;
+  }
+
+  public TransportClientFactory createClientFactory() {
+    return new TransportClientFactory(this);
+  }
+
+  public TransportResponseHandler initializePipeline(
+      SocketChannel channel, ChannelInboundHandlerAdapter decoder) {
+    TransportResponseHandler responseHandler = new TransportResponseHandler(channel);
+    channel
+        .pipeline()
+        .addLast("encoder", ENCODER) // out
+        .addLast("decoder", decoder) // in

Review Comment:
   This seems doesn't align? 
   Why couldn't we create decoder ealier?



##########
common/src/main/java/org/apache/uniffle/common/netty/client/TransportClient.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.uniffle.common.netty.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.SocketAddress;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.common.base.Preconditions;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.util.concurrent.Future;
+import io.netty.util.concurrent.GenericFutureListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.uniffle.common.netty.handle.TransportResponseHandler;
+import org.apache.uniffle.common.netty.protocol.Message;
+import org.apache.uniffle.common.util.NettyUtils;
+
+
+public class TransportClient implements Closeable {
+  private static final Logger logger = LoggerFactory.getLogger(TransportClient.class);
+
+  private Channel channel;
+  private TransportResponseHandler handler;
+  private volatile boolean timedOut;
+
+  private static final AtomicLong counter = new AtomicLong();
+
+  public TransportClient(Channel channel, TransportResponseHandler handler) {
+    this.channel = Preconditions.checkNotNull(channel);
+    this.handler = Preconditions.checkNotNull(handler);
+    this.timedOut = false;
+  }
+
+  public Channel getChannel() {
+    return channel;
+  }
+
+  public boolean isActive() {
+    return !timedOut && (channel.isOpen() || channel.isActive());
+  }
+
+  public SocketAddress getSocketAddress() {
+    return channel.remoteAddress();
+  }
+
+  public ChannelFuture sendShuffleData(Message message, RpcResponseCallback callback) {
+    if (logger.isTraceEnabled()) {
+      logger.trace("Pushing data to {}", NettyUtils.getRemoteAddress(channel));
+    }
+    long requestId = requestId();
+    handler.addResponseCallback(requestId, callback);
+    RpcChannelListener listener = new RpcChannelListener(requestId, callback);
+    return channel.writeAndFlush(message).addListener(listener);
+  }
+
+  public static long requestId() {
+    return counter.getAndIncrement();
+  }
+
+  public class StdChannelListener implements GenericFutureListener<Future<? super Void>> {
+    final long startTime;
+    final Object requestId;
+
+    public StdChannelListener(Object requestId) {
+      this.startTime = System.currentTimeMillis();
+      this.requestId = requestId;
+    }
+
+    @Override
+    public void operationComplete(Future<? super Void> future) throws Exception {
+      if (future.isSuccess()) {
+        if (logger.isTraceEnabled()) {
+          long timeTaken = System.currentTimeMillis() - startTime;
+          logger.trace(
+              "Sending request {} to {} took {} ms",
+              requestId,
+              NettyUtils.getRemoteAddress(channel),
+              timeTaken);
+        }
+      } else {
+        String errorMsg =
+            String.format(
+                "Failed to send request %s to %s: %s, channel will be closed",
+                requestId, NettyUtils.getRemoteAddress(channel), future.cause());
+        logger.warn(errorMsg);
+        channel.close();
+        try {
+          handleFailure(errorMsg, future.cause());
+        } catch (Exception e) {
+          logger.error("Uncaught exception in RPC response callback handler!", e);
+        }
+      }
+    }
+
+    protected void handleFailure(String errorMsg, Throwable cause) {
+      logger.error("Error encountered " + errorMsg, cause);
+    }
+  }
+
+  private class RpcChannelListener extends StdChannelListener {
+    final long rpcRequestId;
+    final RpcResponseCallback callback;
+
+    RpcChannelListener(long rpcRequestId, RpcResponseCallback callback) {
+      super("RPC " + rpcRequestId);
+      this.rpcRequestId = rpcRequestId;
+      this.callback = callback;
+    }
+
+    @Override
+    protected void handleFailure(String errorMsg, Throwable cause) {
+      handler.removeRpcRequest(rpcRequestId);
+      callback.onFailure(new IOException(errorMsg, cause));

Review Comment:
   Still we add a logging here?



-- 
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: issues-unsubscribe@uniffle.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@uniffle.apache.org
For additional commands, e-mail: issues-help@uniffle.apache.org