jerqi commented on code in PR #742: URL: https://github.com/apache/incubator-uniffle/pull/742#discussion_r1142425988
########## common/src/main/java/org/apache/uniffle/common/netty/protocol/Encoders.java: ########## @@ -0,0 +1,69 @@ +/* + * 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.protocol; + +import java.util.List; + +import io.netty.buffer.ByteBuf; + +import org.apache.uniffle.common.ShuffleBlockInfo; +import org.apache.uniffle.common.ShuffleServerInfo; +import org.apache.uniffle.common.util.ByteBufUtils; + +public class Encoders { + public static void encodeShuffleServerInfo(ShuffleServerInfo shuffleServerInfo, ByteBuf byteBuf) { + ByteBufUtils.writeLengthAndString(byteBuf, shuffleServerInfo.getId()); + ByteBufUtils.writeLengthAndString(byteBuf, shuffleServerInfo.getHost()); + byteBuf.writeInt(shuffleServerInfo.getGrpcPort()); + byteBuf.writeInt(shuffleServerInfo.getNettyPort()); + } + + public static void encodeShuffleBlockInfo(ShuffleBlockInfo shuffleBlockInfo, ByteBuf byteBuf) { + byteBuf.writeInt(shuffleBlockInfo.getPartitionId()); + byteBuf.writeLong(shuffleBlockInfo.getBlockId()); + byteBuf.writeInt(shuffleBlockInfo.getLength()); + byteBuf.writeInt(shuffleBlockInfo.getShuffleId()); + byteBuf.writeLong(shuffleBlockInfo.getCrc()); + byteBuf.writeLong(shuffleBlockInfo.getTaskAttemptId()); + ByteBufUtils.copyByteBuf(shuffleBlockInfo.getData(), byteBuf); Review Comment: Do we need to release memory after copying here? ``` shuffleBlockInfo.getData().release(). ``` ########## common/src/main/java/org/apache/uniffle/common/netty/protocol/SendShuffleDataRequest.java: ########## @@ -0,0 +1,153 @@ +/* + * 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.protocol; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import io.netty.buffer.ByteBuf; + +import org.apache.uniffle.common.ShuffleBlockInfo; +import org.apache.uniffle.common.util.ByteBufUtils; + +public class SendShuffleDataRequest extends Message { + public long requestId; + private String appId; + private int shuffleId; + private long requireId; + private Map<Integer, List<ShuffleBlockInfo>> partitionToBlocks; + private long timestamp; + + public SendShuffleDataRequest(long requestId, String appId, int shuffleId, long requireId, + Map<Integer, List<ShuffleBlockInfo>> partitionToBlocks, long timestamp) { + this.requestId = requestId; + this.appId = appId; + this.shuffleId = shuffleId; + this.requireId = requireId; + this.partitionToBlocks = partitionToBlocks; + this.timestamp = timestamp; + } + + @Override + public Type type() { + return Type.SEND_SHUFFLE_DATA_REQUEST; + } + + @Override + public int encodedLength() { + int encodeLength = Long.BYTES + ByteBufUtils.encodedLength(appId) + Integer.BYTES + Long.BYTES + Integer.BYTES; + for (Map.Entry<Integer, List<ShuffleBlockInfo>> entry : partitionToBlocks.entrySet()) { + encodeLength += 2 * Integer.BYTES; + for (ShuffleBlockInfo sbi : entry.getValue()) { + encodeLength += Encoders.encodeLengthOfShuffleBlockInfo(sbi); + } + } + return encodeLength + Long.BYTES; + } + + @Override + public void encode(ByteBuf buf) { + buf.writeLong(requestId); + ByteBufUtils.writeLengthAndString(buf, appId); + buf.writeInt(shuffleId); + buf.writeLong(requireId); + encodePartitionData(buf); + buf.writeLong(timestamp); + } + + private static Map<Integer, List<ShuffleBlockInfo>> decodePartitionData(ByteBuf byteBuf) { + Map<Integer, List<ShuffleBlockInfo>> partitionToBlocks = Maps.newHashMap(); + int lengthOfPartitionData = byteBuf.readInt(); + for (int i = 0; i < lengthOfPartitionData; i++) { + int partitionId = byteBuf.readInt(); + int lengthOfShuffleBlocks = byteBuf.readInt(); + List<ShuffleBlockInfo> shuffleBlockInfoList = Lists.newArrayList(); + for (int j = 0; j < lengthOfShuffleBlocks; j++) { + shuffleBlockInfoList.add(Decoders.decodeShuffleBlockInfo(byteBuf)); + } + partitionToBlocks.put(partitionId, shuffleBlockInfoList); + } + return partitionToBlocks; + } + + public static SendShuffleDataRequest decode(ByteBuf byteBuf) { + long requestId = byteBuf.readLong(); + String appId = ByteBufUtils.readLengthAndString(byteBuf); + int shuffleId = byteBuf.readInt(); + long requireId = byteBuf.readLong(); + Map<Integer, List<ShuffleBlockInfo>> partitionToBlocks = decodePartitionData(byteBuf); + long timestamp = byteBuf.readLong(); + return new SendShuffleDataRequest(requestId, appId, shuffleId, requireId, partitionToBlocks, timestamp); + } + + private void encodePartitionData(ByteBuf buf) { + buf.writeInt(partitionToBlocks.size()); + for (Map.Entry<Integer, List<ShuffleBlockInfo>> entry : partitionToBlocks.entrySet()) { + buf.writeInt(entry.getKey()); + buf.writeInt(entry.getValue().size()); + for (ShuffleBlockInfo sbi : entry.getValue()) { + Encoders.encodeShuffleBlockInfo(sbi, buf); + } + } + } + + public long getRequestId() { + return requestId; + } + + public String getAppId() { + return appId; + } + + public int getShuffleId() { + return shuffleId; + } + + public long getRequireId() { + return requireId; + } + + public Map<Integer, List<ShuffleBlockInfo>> getPartitionToBlocks() { + return partitionToBlocks; + } + + @Override + public boolean equals(Object o) { Review Comment: Why do we need to override the method ``equal? ########## common/src/main/java/org/apache/uniffle/common/netty/MessageEncoder.java: ########## @@ -0,0 +1,49 @@ +/* + * 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; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelOutboundHandlerAdapter; +import io.netty.channel.ChannelPromise; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.uniffle.common.netty.protocol.Message; + +public class MessageEncoder extends ChannelOutboundHandlerAdapter { + + private static final Logger LOG = LoggerFactory.getLogger(MessageEncoder.class); + + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) { + // todo: support zero copy + Message message = (Message) msg; + int encodeLength = message.encodedLength(); + ByteBuf byteBuf = ctx.alloc().buffer(FrameDecoder.HEADER_SIZE + encodeLength); + try { + byteBuf.writeInt(encodeLength); + byteBuf.writeByte(message.type().id()); + message.encode(byteBuf); + } catch (Exception e) { + LOG.error("Unexpected exception during process encode!", e); + byteBuf.release(); + } + ctx.writeAndFlush(byteBuf); Review Comment: When do we release byteBuf? -- 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] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
