This is an automated email from the ASF dual-hosted git repository.

maobaolong pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-uniffle.git


The following commit(s) were added to refs/heads/master by this push:
     new 680f96a19 [#2181] Improvement[server] Replace ShuffleBlockInfo with 
ShufflePartitionedBlock with netty mode. (#2182)
680f96a19 is described below

commit 680f96a195884c1d484a0602a4bd0a116e5376be
Author: leewish <[email protected]>
AuthorDate: Thu Oct 31 14:37:50 2024 +0800

    [#2181] Improvement[server] Replace ShuffleBlockInfo with 
ShufflePartitionedBlock with netty mode. (#2182)
    
    ### What changes were proposed in this pull request?
    
    On the server side of the netty mode, when the server receives the 
sendShuffleData message, it decodes it into an unnecessary ShuffleBlockInfo and 
then converts it to ShufflePartitionedBlock in handleSendShuffleDataRequest.
    
    ### Why are the changes needed?
    
    Fix: #2181
    
    ### Does this PR introduce _any_ user-facing change?
    
    No.
    
    ### How was this patch tested?
    Locally
    
    Co-authored-by: wenlongwlli <[email protected]>
---
 .../uniffle/common/netty/protocol/Decoders.java    |  27 ++++
 .../uniffle/common/netty/protocol/Message.java     |   2 +-
 .../netty/protocol/SendShuffleDataRequestV1.java   | 149 +++++++++++++++++++++
 .../common/netty/protocol/NettyProtocolTest.java   |  11 +-
 .../netty/protocol/NettyProtocolTestUtils.java     |  64 ++++++++-
 .../server/netty/ShuffleServerNettyHandler.java    |  36 +++--
 6 files changed, 256 insertions(+), 33 deletions(-)

diff --git 
a/common/src/main/java/org/apache/uniffle/common/netty/protocol/Decoders.java 
b/common/src/main/java/org/apache/uniffle/common/netty/protocol/Decoders.java
index fc7004880..f6dea2226 100644
--- 
a/common/src/main/java/org/apache/uniffle/common/netty/protocol/Decoders.java
+++ 
b/common/src/main/java/org/apache/uniffle/common/netty/protocol/Decoders.java
@@ -26,6 +26,7 @@ import io.netty.buffer.ByteBuf;
 
 import org.apache.uniffle.common.BufferSegment;
 import org.apache.uniffle.common.ShuffleBlockInfo;
+import org.apache.uniffle.common.ShufflePartitionedBlock;
 import org.apache.uniffle.common.ShuffleServerInfo;
 import org.apache.uniffle.common.util.ByteBufUtils;
 import org.apache.uniffle.common.util.NettyUtils;
@@ -69,6 +70,32 @@ public class Decoders {
         taskAttemptId);
   }
 
+  public static ShufflePartitionedBlock 
decodeShufflePartitionedBlockV1(ByteBuf byteBuf) {
+    // To ensure upgrade compatibility, only the server-side code was modified.
+    // Some unused information will be skipped during decoding.
+    // TODO: Remove the ShuffleBlockInfo used in the communication between 
client and server via
+    // handleSendShuffleDataRequest.
+    //    https://github.com/apache/incubator-uniffle/issues/2201
+    byteBuf.skipBytes(4); // partId Int
+    final long blockId = byteBuf.readLong();
+    final int length = byteBuf.readInt();
+    byteBuf.skipBytes(4); // shuffleId Int
+    final long crc = byteBuf.readLong();
+    final long taskAttemptId = byteBuf.readLong();
+    int dataLength = byteBuf.readInt();
+    ByteBuf data = 
NettyUtils.getSharedUnpooledByteBufAllocator(true).directBuffer(dataLength);
+    data.writeBytes(byteBuf, dataLength);
+    int lengthOfShuffleServers = byteBuf.readInt();
+    List<ShuffleServerInfo> serverInfos = Lists.newArrayList();
+    for (int k = 0; k < lengthOfShuffleServers; k++) {
+      serverInfos.add(decodeShuffleServerInfo(byteBuf));
+    }
+    final int uncompressLength = byteBuf.readInt();
+    byteBuf.skipBytes(8); // freeMemory Long
+
+    return new ShufflePartitionedBlock(length, uncompressLength, crc, blockId, 
taskAttemptId, data);
+  }
+
   public static Map<Integer, List<Long>> decodePartitionToBlockIds(ByteBuf 
byteBuf) {
     Map<Integer, List<Long>> partitionToBlockIds = Maps.newHashMap();
     int mapSize = byteBuf.readInt();
diff --git 
a/common/src/main/java/org/apache/uniffle/common/netty/protocol/Message.java 
b/common/src/main/java/org/apache/uniffle/common/netty/protocol/Message.java
index ed5167e77..e7f18ab2f 100644
--- a/common/src/main/java/org/apache/uniffle/common/netty/protocol/Message.java
+++ b/common/src/main/java/org/apache/uniffle/common/netty/protocol/Message.java
@@ -151,7 +151,7 @@ public abstract class Message implements Encodable {
       case RPC_RESPONSE:
         return RpcResponse.decode(in, false);
       case SEND_SHUFFLE_DATA_REQUEST:
-        return SendShuffleDataRequest.decode(in);
+        return SendShuffleDataRequestV1.decode(in);
       case GET_LOCAL_SHUFFLE_DATA_REQUEST:
         return GetLocalShuffleDataRequest.decode(in);
       case GET_LOCAL_SHUFFLE_DATA_RESPONSE:
diff --git 
a/common/src/main/java/org/apache/uniffle/common/netty/protocol/SendShuffleDataRequestV1.java
 
b/common/src/main/java/org/apache/uniffle/common/netty/protocol/SendShuffleDataRequestV1.java
new file mode 100644
index 000000000..fe79e2281
--- /dev/null
+++ 
b/common/src/main/java/org/apache/uniffle/common/netty/protocol/SendShuffleDataRequestV1.java
@@ -0,0 +1,149 @@
+/*
+ * 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 com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+
+import org.apache.uniffle.common.ShufflePartitionedBlock;
+import org.apache.uniffle.common.util.ByteBufUtils;
+
+public class SendShuffleDataRequestV1 extends RequestMessage {
+  private String appId;
+  private int shuffleId;
+
+  private int stageAttemptNumber;
+  private long requireId;
+  private Map<Integer, List<ShufflePartitionedBlock>> partitionToBlocks;
+  private long timestamp;
+  private int decodedLength;
+
+  public SendShuffleDataRequestV1(long requestId) {
+    super(requestId);
+  }
+
+  @Override
+  public Type type() {
+    return Type.SEND_SHUFFLE_DATA_REQUEST;
+  }
+
+  @Override
+  public int encodedLength() {
+    return -1;
+  }
+
+  @Override
+  public void encode(ByteBuf buf) {}
+
+  public void decodeShuffleData(ByteBuf byteBuf) {
+    final int startIndex = byteBuf.readerIndex();
+    this.appId = ByteBufUtils.readLengthAndString(byteBuf);
+    this.shuffleId = byteBuf.readInt();
+    this.requireId = byteBuf.readLong();
+    this.partitionToBlocks = decodePartitionData(byteBuf);
+    this.timestamp = byteBuf.readLong();
+    int endIndex = byteBuf.readerIndex();
+    decodedLength += endIndex - startIndex;
+  }
+
+  public int getDecodedLength() {
+    return decodedLength;
+  }
+
+  private Map<Integer, List<ShufflePartitionedBlock>> 
decodePartitionData(ByteBuf byteBuf) {
+    Map<Integer, List<ShufflePartitionedBlock>> partitionToBlocks = 
Maps.newHashMap();
+    int lengthOfPartitionData = byteBuf.readInt();
+    for (int i = 0; i < lengthOfPartitionData; i++) {
+      int partitionId = byteBuf.readInt();
+      int lengthOfShuffleBlocks = byteBuf.readInt();
+      List<ShufflePartitionedBlock> shufflePartitionedBlocks = 
Lists.newArrayList();
+      for (int j = 0; j < lengthOfShuffleBlocks; j++) {
+        try {
+          
shufflePartitionedBlocks.add(Decoders.decodeShufflePartitionedBlockV1(byteBuf));
+        } catch (Throwable t) {
+          shufflePartitionedBlocks.forEach(sbi -> sbi.getData().release());
+          if (!partitionToBlocks.isEmpty()) {
+            partitionToBlocks.forEach(
+                (integer, shuffleBlockInfos) -> {
+                  shuffleBlockInfos.forEach(sbi -> sbi.getData().release());
+                });
+          }
+          throw t;
+        }
+      }
+      partitionToBlocks.put(partitionId, shufflePartitionedBlocks);
+    }
+    return partitionToBlocks;
+  }
+
+  public static SendShuffleDataRequestV1 decode(ByteBuf byteBuf) {
+    int startIndex = byteBuf.readerIndex();
+    long requestId = byteBuf.readLong();
+    SendShuffleDataRequestV1 req = new SendShuffleDataRequestV1(requestId);
+    req.decodeShuffleData(byteBuf);
+    int endIndex = byteBuf.readerIndex();
+    req.setDecodedLength(endIndex - startIndex);
+
+    return req;
+  }
+
+  public String getAppId() {
+    return appId;
+  }
+
+  public int getShuffleId() {
+    return shuffleId;
+  }
+
+  public long getRequireId() {
+    return requireId;
+  }
+
+  public void setRequireId(long requireId) {
+    this.requireId = requireId;
+  }
+
+  public Map<Integer, List<ShufflePartitionedBlock>> getPartitionToBlocks() {
+    return partitionToBlocks;
+  }
+
+  public long getTimestamp() {
+    return timestamp;
+  }
+
+  public void setTimestamp(long timestamp) {
+    this.timestamp = timestamp;
+  }
+
+  public int getStageAttemptNumber() {
+    return stageAttemptNumber;
+  }
+
+  @Override
+  public String getOperationType() {
+    return "sendShuffleDataV1";
+  }
+
+  public void setDecodedLength(int decodedLength) {
+    this.decodedLength = decodedLength;
+  }
+}
diff --git 
a/common/src/test/java/org/apache/uniffle/common/netty/protocol/NettyProtocolTest.java
 
b/common/src/test/java/org/apache/uniffle/common/netty/protocol/NettyProtocolTest.java
index 072985d09..276165d96 100644
--- 
a/common/src/test/java/org/apache/uniffle/common/netty/protocol/NettyProtocolTest.java
+++ 
b/common/src/test/java/org/apache/uniffle/common/netty/protocol/NettyProtocolTest.java
@@ -30,6 +30,7 @@ import org.roaringbitmap.longlong.Roaring64NavigableMap;
 
 import org.apache.uniffle.common.BufferSegment;
 import org.apache.uniffle.common.ShuffleBlockInfo;
+import org.apache.uniffle.common.ShufflePartitionedBlock;
 import org.apache.uniffle.common.ShuffleServerInfo;
 import org.apache.uniffle.common.netty.buffer.ManagedBuffer;
 import org.apache.uniffle.common.netty.buffer.NettyManagedBuffer;
@@ -103,17 +104,17 @@ public class NettyProtocolTest {
     ByteBuf byteBuf = Unpooled.buffer(sendShuffleDataRequest.encodedLength());
     sendShuffleDataRequest.encode(byteBuf);
     assertEquals(byteBuf.readableBytes(), encodeLength);
-    SendShuffleDataRequest sendShuffleDataRequest1 = 
sendShuffleDataRequest.decode(byteBuf);
+    SendShuffleDataRequestV1 sendShuffleDataRequest1 = 
SendShuffleDataRequestV1.decode(byteBuf);
     assertTrue(
-        NettyProtocolTestUtils.compareSendShuffleDataRequest(
+        NettyProtocolTestUtils.compareSendShuffleDataRequestV1(
             sendShuffleDataRequest, sendShuffleDataRequest1));
-    assertEquals(encodeLength, sendShuffleDataRequest1.encodedLength());
+    assertEquals(encodeLength, sendShuffleDataRequest1.getDecodedLength());
     byteBuf.release();
-    for (ShuffleBlockInfo shuffleBlockInfo :
+    for (ShufflePartitionedBlock shuffleBlockInfo :
         sendShuffleDataRequest1.getPartitionToBlocks().get(1)) {
       shuffleBlockInfo.getData().release();
     }
-    for (ShuffleBlockInfo shuffleBlockInfo :
+    for (ShufflePartitionedBlock shuffleBlockInfo :
         sendShuffleDataRequest1.getPartitionToBlocks().get(2)) {
       shuffleBlockInfo.getData().release();
     }
diff --git 
a/common/src/test/java/org/apache/uniffle/common/netty/protocol/NettyProtocolTestUtils.java
 
b/common/src/test/java/org/apache/uniffle/common/netty/protocol/NettyProtocolTestUtils.java
index e4891aa5a..3ce237cd2 100644
--- 
a/common/src/test/java/org/apache/uniffle/common/netty/protocol/NettyProtocolTestUtils.java
+++ 
b/common/src/test/java/org/apache/uniffle/common/netty/protocol/NettyProtocolTestUtils.java
@@ -22,21 +22,28 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.uniffle.common.ShuffleBlockInfo;
+import org.apache.uniffle.common.ShufflePartitionedBlock;
 
 public class NettyProtocolTestUtils {
 
   private static boolean compareShuffleBlockInfo(
       ShuffleBlockInfo blockInfo1, ShuffleBlockInfo blockInfo2) {
-    return blockInfo1.getPartitionId() == blockInfo2.getPartitionId()
-        && blockInfo1.getBlockId() == blockInfo2.getBlockId()
+    return blockInfo1.getBlockId() == blockInfo2.getBlockId()
         && blockInfo1.getLength() == blockInfo2.getLength()
-        && blockInfo1.getShuffleId() == blockInfo2.getShuffleId()
         && blockInfo1.getCrc() == blockInfo2.getCrc()
         && blockInfo1.getTaskAttemptId() == blockInfo2.getTaskAttemptId()
         && blockInfo1.getUncompressLength() == blockInfo2.getUncompressLength()
-        && blockInfo1.getFreeMemory() == blockInfo2.getFreeMemory()
-        && blockInfo1.getData().equals(blockInfo2.getData())
-        && 
blockInfo1.getShuffleServerInfos().equals(blockInfo2.getShuffleServerInfos());
+        && blockInfo1.getData().equals(blockInfo2.getData());
+  }
+
+  private static boolean compareShuffleBlockInfoV1(
+      ShuffleBlockInfo blockInfo1, ShufflePartitionedBlock blockInfo2) {
+    return blockInfo1.getBlockId() == blockInfo2.getBlockId()
+        && blockInfo1.getLength() == blockInfo2.getDataLength()
+        && blockInfo1.getCrc() == blockInfo2.getCrc()
+        && blockInfo1.getTaskAttemptId() == blockInfo2.getTaskAttemptId()
+        && blockInfo1.getUncompressLength() == blockInfo2.getUncompressLength()
+        && blockInfo1.getData().equals(blockInfo2.getData());
   }
 
   private static boolean compareBlockList(
@@ -52,6 +59,19 @@ public class NettyProtocolTestUtils {
     return true;
   }
 
+  private static boolean compareBlockListV1(
+      List<ShuffleBlockInfo> list1, List<ShufflePartitionedBlock> list2) {
+    if (list1 == null || list2 == null || list1.size() != list2.size()) {
+      return false;
+    }
+    for (int i = 0; i < list1.size(); i++) {
+      if (!compareShuffleBlockInfoV1(list1.get(i), list2.get(i))) {
+        return false;
+      }
+    }
+    return true;
+  }
+
   private static boolean comparePartitionToBlockList(
       Map<Integer, List<ShuffleBlockInfo>> m1, Map<Integer, 
List<ShuffleBlockInfo>> m2) {
     if (m1 == null || m2 == null || m1.size() != m2.size()) {
@@ -67,6 +87,21 @@ public class NettyProtocolTestUtils {
     return true;
   }
 
+  private static boolean comparePartitionToBlockListV1(
+      Map<Integer, List<ShuffleBlockInfo>> m1, Map<Integer, 
List<ShufflePartitionedBlock>> m2) {
+    if (m1 == null || m2 == null || m1.size() != m2.size()) {
+      return false;
+    }
+    Iterator<Map.Entry<Integer, List<ShuffleBlockInfo>>> iter1 = 
m1.entrySet().iterator();
+    while (iter1.hasNext()) {
+      Map.Entry<Integer, List<ShuffleBlockInfo>> entry1 = iter1.next();
+      if (!compareBlockListV1(entry1.getValue(), m2.get(entry1.getKey()))) {
+        return false;
+      }
+    }
+    return true;
+  }
+
   public static boolean compareSendShuffleDataRequest(
       SendShuffleDataRequest req1, SendShuffleDataRequest req2) {
     if (req1 == req2) {
@@ -86,4 +121,21 @@ public class NettyProtocolTestUtils {
     }
     return comparePartitionToBlockList(req1.getPartitionToBlocks(), 
req2.getPartitionToBlocks());
   }
+
+  public static boolean compareSendShuffleDataRequestV1(
+      SendShuffleDataRequest req1, SendShuffleDataRequestV1 req2) {
+    if (req1 == null || req2 == null) {
+      return false;
+    }
+    boolean isEqual =
+        req1.getRequestId() == req2.getRequestId()
+            && req1.getShuffleId() == req2.getShuffleId()
+            && req1.getRequireId() == req2.getRequireId()
+            && req1.getTimestamp() == req2.getTimestamp()
+            && req1.getAppId().equals(req2.getAppId());
+    if (!isEqual) {
+      return false;
+    }
+    return comparePartitionToBlockListV1(req1.getPartitionToBlocks(), 
req2.getPartitionToBlocks());
+  }
 }
diff --git 
a/server/src/main/java/org/apache/uniffle/server/netty/ShuffleServerNettyHandler.java
 
b/server/src/main/java/org/apache/uniffle/server/netty/ShuffleServerNettyHandler.java
index eb1951a5f..b5137d818 100644
--- 
a/server/src/main/java/org/apache/uniffle/server/netty/ShuffleServerNettyHandler.java
+++ 
b/server/src/main/java/org/apache/uniffle/server/netty/ShuffleServerNettyHandler.java
@@ -35,7 +35,6 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.uniffle.common.BufferSegment;
 import org.apache.uniffle.common.ReconfigurableRegistry;
-import org.apache.uniffle.common.ShuffleBlockInfo;
 import org.apache.uniffle.common.ShuffleDataResult;
 import org.apache.uniffle.common.ShuffleIndexResult;
 import org.apache.uniffle.common.ShufflePartitionedBlock;
@@ -61,6 +60,7 @@ import 
org.apache.uniffle.common.netty.protocol.GetSortedShuffleDataResponse;
 import org.apache.uniffle.common.netty.protocol.RequestMessage;
 import org.apache.uniffle.common.netty.protocol.RpcResponse;
 import org.apache.uniffle.common.netty.protocol.SendShuffleDataRequest;
+import org.apache.uniffle.common.netty.protocol.SendShuffleDataRequestV1;
 import org.apache.uniffle.common.rpc.StatusCode;
 import org.apache.uniffle.server.ShuffleDataReadEvent;
 import org.apache.uniffle.server.ShuffleServer;
@@ -120,8 +120,8 @@ public class ShuffleServerNettyHandler implements 
BaseMessageHandler {
   @Override
   public void receive(TransportClient client, RequestMessage msg) {
     shuffleServer.getNettyMetrics().incCounter(msg.getClass().getName());
-    if (msg instanceof SendShuffleDataRequest) {
-      handleSendShuffleDataRequest(client, (SendShuffleDataRequest) msg);
+    if (msg instanceof SendShuffleDataRequestV1) {
+      handleSendShuffleDataRequest(client, (SendShuffleDataRequestV1) msg);
     } else if (msg instanceof GetLocalShuffleDataRequest) {
       handleGetLocalShuffleData(client, (GetLocalShuffleDataRequest) msg);
     } else if (msg instanceof GetLocalShuffleIndexRequest) {
@@ -141,7 +141,7 @@ public class ShuffleServerNettyHandler implements 
BaseMessageHandler {
     LOG.error("exception caught {}", client.getSocketAddress(), cause);
   }
 
-  public void handleSendShuffleDataRequest(TransportClient client, 
SendShuffleDataRequest req) {
+  public void handleSendShuffleDataRequest(TransportClient client, 
SendShuffleDataRequestV1 req) {
     try (ServerRpcAuditContext auditContext = 
createAuditContext("sendShuffleData", client)) {
       RpcResponse rpcResponse;
       String appId = req.getAppId();
@@ -156,8 +156,7 @@ public class ShuffleServerNettyHandler implements 
BaseMessageHandler {
       PreAllocatedBufferInfo info =
           shuffleTaskManager.getAndRemovePreAllocatedBuffer(requireBufferId);
       int requireSize = info == null ? 0 : info.getRequireSize();
-      int requireBlocksSize =
-          requireSize - req.encodedLength() < 0 ? 0 : requireSize - 
req.encodedLength();
+      int requireBlocksSize = Math.max(requireSize - req.getDecodedLength(), 
0);
 
       boolean isPreAllocated = info != null;
 
@@ -280,7 +279,7 @@ public class ShuffleServerNettyHandler implements 
BaseMessageHandler {
           return;
         }
         final long start = System.currentTimeMillis();
-        shuffleBufferManager.releaseMemory(req.encodedLength(), false, true);
+        shuffleBufferManager.releaseMemory(req.getDecodedLength(), false, 
true);
         List<ShufflePartitionedData> shufflePartitionedDataList = 
toPartitionedDataList(req);
         long alreadyReleasedSize = 0;
         boolean hasFailureOccurred = false;
@@ -384,7 +383,7 @@ public class ShuffleServerNettyHandler implements 
BaseMessageHandler {
   }
 
   private static void releaseNettyBufferAndMetrics(
-      SendShuffleDataRequest req,
+      SendShuffleDataRequestV1 req,
       String appId,
       int shuffleId,
       long requireBufferId,
@@ -863,32 +862,27 @@ public class ShuffleServerNettyHandler implements 
BaseMessageHandler {
     }
   }
 
-  private List<ShufflePartitionedData> 
toPartitionedDataList(SendShuffleDataRequest req) {
+  private List<ShufflePartitionedData> 
toPartitionedDataList(SendShuffleDataRequestV1 req) {
     List<ShufflePartitionedData> ret = Lists.newArrayList();
 
-    for (Map.Entry<Integer, List<ShuffleBlockInfo>> entry : 
req.getPartitionToBlocks().entrySet()) {
+    for (Map.Entry<Integer, List<ShufflePartitionedBlock>> entry :
+        req.getPartitionToBlocks().entrySet()) {
       ret.add(toPartitionedData(entry.getKey(), entry.getValue()));
     }
     return ret;
   }
 
-  private ShufflePartitionedData toPartitionedData(int partitionId, 
List<ShuffleBlockInfo> blocks) {
-    if (blocks == null || blocks.size() == 0) {
+  private ShufflePartitionedData toPartitionedData(
+      int partitionId, List<ShufflePartitionedBlock> blocks) {
+    if (blocks == null || blocks.isEmpty()) {
       return new ShufflePartitionedData(partitionId, 0L, 0L, new 
ShufflePartitionedBlock[] {});
     }
     ShufflePartitionedBlock[] ret = new ShufflePartitionedBlock[blocks.size()];
     long encodedLength = 0L;
     long dataLength = 0L;
     int i = 0;
-    for (ShuffleBlockInfo block : blocks) {
-      ret[i] =
-          new ShufflePartitionedBlock(
-              block.getLength(),
-              block.getUncompressLength(),
-              block.getCrc(),
-              block.getBlockId(),
-              block.getTaskAttemptId(),
-              block.getData());
+    for (ShufflePartitionedBlock block : blocks) {
+      ret[i] = block;
       encodedLength += ret[i].getEncodedLength();
       dataLength += ret[i].getDataLength();
       i++;

Reply via email to