You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@celeborn.apache.org by "RexXiong (via GitHub)" <gi...@apache.org> on 2023/02/15 09:06:48 UTC

[GitHub] [incubator-celeborn] RexXiong commented on a diff in pull request #1222: [CELEBORN-283] Derive network layer for flink plugin.

RexXiong commented on code in PR #1222:
URL: https://github.com/apache/incubator-celeborn/pull/1222#discussion_r1106834389


##########
client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/network/MessageDecoderExt.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.celeborn.plugin.flink.network;
+
+import java.nio.charset.StandardCharsets;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import org.apache.celeborn.common.network.buffer.NettyManagedBuffer;
+import org.apache.celeborn.common.network.protocol.*;
+import org.apache.celeborn.plugin.flink.buffer.FlinkNettyManagedBuffer;
+import org.apache.celeborn.plugin.flink.protocol.ReadData;
+
+public class MessageDecoderExt {
+  public static Message decode(Message.Type type, ByteBuf in, boolean decodeBody) {
+    long requestId;
+    switch (type) {
+      case RPC_REQUEST:
+        requestId = in.readLong();
+        // See comment in encodedLength().
+        in.readInt();

Review Comment:
   better to replace this with link the actual class use {@link class_identifier#method}



##########
client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/network/MapTransportClientFactory.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.celeborn.plugin.flink.network;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.socket.SocketChannel;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.celeborn.common.network.TransportContext;
+import org.apache.celeborn.common.network.client.TransportClient;
+import org.apache.celeborn.common.network.client.TransportClientFactory;
+import org.apache.celeborn.common.network.server.TransportChannelHandler;
+
+public class MapTransportClientFactory extends TransportClientFactory {
+  private static final Logger logger = LoggerFactory.getLogger(MapTransportClientFactory.class);
+  protected final MapTransportContext mapTransportContext;
+
+  public MapTransportClientFactory(MapTransportContext context) {

Review Comment:
   better rename MapTransportClientFactory to FlinkTransportClientFactory



##########
client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/network/MapTransportContext.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.celeborn.plugin.flink.network;
+
+import java.util.function.Supplier;
+
+import io.netty.channel.Channel;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.handler.timeout.IdleStateHandler;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.celeborn.common.network.TransportContext;
+import org.apache.celeborn.common.network.client.TransportClient;
+import org.apache.celeborn.common.network.client.TransportResponseHandler;
+import org.apache.celeborn.common.network.server.BaseMessageHandler;
+import org.apache.celeborn.common.network.server.TransportChannelHandler;
+import org.apache.celeborn.common.network.server.TransportRequestHandler;
+import org.apache.celeborn.common.network.util.FrameDecoder;
+import org.apache.celeborn.common.network.util.TransportConf;
+import org.apache.celeborn.plugin.flink.utils.Utils;
+
+public class MapTransportContext extends TransportContext {

Review Comment:
   MapTransportContext -> FlinkTransportContext



##########
client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/network/MessageDecoderExt.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.celeborn.plugin.flink.network;
+
+import java.nio.charset.StandardCharsets;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import org.apache.celeborn.common.network.buffer.NettyManagedBuffer;
+import org.apache.celeborn.common.network.protocol.*;
+import org.apache.celeborn.plugin.flink.buffer.FlinkNettyManagedBuffer;
+import org.apache.celeborn.plugin.flink.protocol.ReadData;
+
+public class MessageDecoderExt {
+  public static Message decode(Message.Type type, ByteBuf in, boolean decodeBody) {
+    long requestId;
+    switch (type) {
+      case RPC_REQUEST:
+        requestId = in.readLong();
+        // See comment in encodedLength().
+        in.readInt();
+        if (decodeBody) {
+          return new RpcRequest(requestId, new FlinkNettyManagedBuffer(in));
+        } else {
+          return new RpcRequest(requestId, NettyManagedBuffer.EmptyBuffer);
+        }
+
+      case RPC_RESPONSE:
+        requestId = in.readLong();
+        // See comment in encodedLength().

Review Comment:
   ditto



##########
client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/network/MapTransportContext.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.celeborn.plugin.flink.network;
+
+import java.util.function.Supplier;
+
+import io.netty.channel.Channel;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.handler.timeout.IdleStateHandler;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.celeborn.common.network.TransportContext;
+import org.apache.celeborn.common.network.client.TransportClient;
+import org.apache.celeborn.common.network.client.TransportResponseHandler;
+import org.apache.celeborn.common.network.server.BaseMessageHandler;
+import org.apache.celeborn.common.network.server.TransportChannelHandler;
+import org.apache.celeborn.common.network.server.TransportRequestHandler;
+import org.apache.celeborn.common.network.util.FrameDecoder;
+import org.apache.celeborn.common.network.util.TransportConf;
+import org.apache.celeborn.plugin.flink.utils.Utils;
+
+public class MapTransportContext extends TransportContext {
+
+  private static final Logger logger = LoggerFactory.getLogger(MapTransportContext.class);
+
+  public MapTransportContext(
+      TransportConf conf, BaseMessageHandler msgHandler, boolean closeIdleConnections) {
+    super(conf, msgHandler, closeIdleConnections);
+  }
+
+  public TransportChannelHandler createChannelHandler(
+      Channel channel, BaseMessageHandler msgHandler) {
+    TransportResponseHandler responseHandler = new TransportResponseHandler(channel);
+    TransportClient client = new TransportClient(channel, responseHandler);
+    TransportRequestHandler requestHandler =
+        new TransportRequestHandler(channel, client, msgHandler);
+    return new TransportChannelHandler(
+        client, responseHandler, requestHandler, conf.connectionTimeoutMs(), closeIdleConnections);
+  }
+
+  public TransportChannelHandler initializePipeline(
+      SocketChannel channel, Supplier<ByteBuf> bufSupplier) {
+    try {
+      if (channelsLimiter != null) {
+        channel.pipeline().addLast("limiter", channelsLimiter);
+      }
+      TransportChannelHandler channelHandler = createChannelHandler(channel, msgHandler);
+      channel
+          .pipeline()
+          .addLast("encoder", ENCODER)
+          .addLast(
+              FrameDecoder.HANDLER_NAME, Utils.createFrameDecoderWithBufferSupplier(bufSupplier))

Review Comment:
   why not use new TransportFrameDecoderWithBufferSupplier(bufSupplier) directly?



##########
client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/network/ReadClientHandler.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.celeborn.plugin.flink.network;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.celeborn.common.network.client.TransportClient;
+import org.apache.celeborn.common.network.protocol.RequestMessage;
+import org.apache.celeborn.common.network.server.BaseMessageHandler;
+import org.apache.celeborn.plugin.flink.protocol.ReadData;
+
+public class ReadClientHandler extends BaseMessageHandler {
+  private static Logger logger = LoggerFactory.getLogger(ReadClientHandler.class);
+  private ConcurrentHashMap<Long, Consumer<RequestMessage>> streamHandlers =
+      new ConcurrentHashMap<>();
+  private ConcurrentHashMap<Long, TransportClient> streamClients = new ConcurrentHashMap<>();
+
+  public void registerHandler(
+      long streamId, Consumer<RequestMessage> handle, TransportClient client) {
+    streamHandlers.put(streamId, handle);
+    streamClients.put(streamId, client);
+  }
+
+  public void removeHandler(long streamId) {
+    streamHandlers.remove(streamId);
+    streamClients.remove(streamId);
+  }
+
+  @Override
+  public void receive(TransportClient client, RequestMessage msg) {
+    long streamId = 0;
+    switch (msg.type()) {
+      case READ_DATA:

Review Comment:
   remove all of this if not ready



-- 
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@celeborn.apache.org

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