FMX commented on code in PR #2979:
URL: https://github.com/apache/celeborn/pull/2979#discussion_r1875559185


##########
common/src/main/java/org/apache/celeborn/common/util/PushDataHeaderUtils.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.common.util;
+
+import java.util.zip.CRC32;
+
+import org.apache.celeborn.common.unsafe.Platform;
+
+public class PushDataHeaderUtils {
+  // Data Header Layout:
+  // | mapId (4 bytes)                        |
+  // | attemptId (4 bytes)                    |
+  // | batchId with checksum flag (4 bytes)   |
+  // | length with checksum length (4 bytes)  |
+  // | checksum (4 bytes)                     |
+  //
+  // Fields description:
+  // - mapId: Unique identifier for the map (4 bytes)
+  // - attemptId: Identifier for the attempt (4 bytes)
+  // - batchId with checksum flag:
+  //   -- checksum flag: 1 bit (indicates if batchId has a checksum)
+  //   -- batchId: 31 bits (always positive when represented as an integer)
+  // - length with checksum length: total length of the data + 4 bytes for 
checksum
+  // - checksum: Always positive integer (4 bytes)
+  public static final int BATCH_HEADER_SIZE = 5 * 4;
+  public static final int BATCH_HEADER_SIZE_WITHOUT_CHECKSUM = 
BATCH_HEADER_SIZE - 4;
+  public static final int MAP_ID_OFFSET = Platform.BYTE_ARRAY_OFFSET;
+  public static final int ATTEMPT_ID_OFFSET = Platform.BYTE_ARRAY_OFFSET + 4;
+  public static final int BATCH_ID_OFFSET = Platform.BYTE_ARRAY_OFFSET + 8;
+  public static final int LENGTH_OFFSET = Platform.BYTE_ARRAY_OFFSET + 12;
+  public static final int CHECKSUM_OFFSET = Platform.BYTE_ARRAY_OFFSET + 16;
+  public static final int POSITIVE_MASK = 0x7FFFFFFF;
+  public static final int HIGHEST_1_BIT_FLAG_MASK = 0x80000000;
+
+  public static void buildDataHeader(
+      byte[] data, int mapId, int attemptId, int batchId, int length) {
+    assert data.length >= BATCH_HEADER_SIZE;
+    int batchIdWithChecksumFlag = batchIdWithChecksumFlag(batchId);
+    int lengthWithChecksum = length + 4;
+    Platform.putInt(data, MAP_ID_OFFSET, mapId);
+    Platform.putInt(data, ATTEMPT_ID_OFFSET, attemptId);
+    Platform.putInt(data, BATCH_ID_OFFSET, batchIdWithChecksumFlag);
+    Platform.putInt(data, LENGTH_OFFSET, lengthWithChecksum);
+    Platform.putInt(data, CHECKSUM_OFFSET, computeHeaderChecksum32(data));
+  }
+
+  public static int batchIdWithChecksumFlag(int batchId) {
+    return batchId | HIGHEST_1_BIT_FLAG_MASK;
+  }
+
+  public static int batchIdWithoutChecksumFlag(int batchId) {
+    return batchId & POSITIVE_MASK;
+  }
+
+  public static boolean hasChecksumFlag(byte[] data) {
+    int batchId = Platform.getInt(data, BATCH_ID_OFFSET);
+    return (batchId & HIGHEST_1_BIT_FLAG_MASK) != 0;
+  }
+
+  public static int getMapId(byte[] data) {
+    return Platform.getInt(data, MAP_ID_OFFSET);
+  }
+
+  public static int getAttemptId(byte[] data) {
+    return Platform.getInt(data, ATTEMPT_ID_OFFSET);
+  }
+
+  public static int getBatchId(byte[] data) {
+    return batchIdWithoutChecksumFlag(Platform.getInt(data, BATCH_ID_OFFSET));
+  }
+
+  public static int getLength(byte[] data) {
+    return Platform.getInt(data, LENGTH_OFFSET) - 4;

Review Comment:
   This will cause errors if an old client reads from Celeborn workers with 
this feature.



##########
client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/readclient/FlinkShuffleClientImpl.java:
##########
@@ -61,10 +61,7 @@
 import org.apache.celeborn.common.protocol.message.ControlMessages;
 import org.apache.celeborn.common.protocol.message.StatusCode;
 import org.apache.celeborn.common.rpc.RpcEndpointRef;
-import org.apache.celeborn.common.util.CollectionUtils;
-import org.apache.celeborn.common.util.JavaUtils;
-import org.apache.celeborn.common.util.PbSerDeUtils;
-import org.apache.celeborn.common.util.Utils;
+import org.apache.celeborn.common.util.*;

Review Comment:
   Irrelevant changes. You can revert them.



##########
common/src/main/java/org/apache/celeborn/common/util/PushDataHeaderUtils.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.common.util;
+
+import java.util.zip.CRC32;
+
+import org.apache.celeborn.common.unsafe.Platform;
+
+public class PushDataHeaderUtils {
+  // Data Header Layout:
+  // | mapId (4 bytes)                        |
+  // | attemptId (4 bytes)                    |
+  // | batchId with checksum flag (4 bytes)   |
+  // | length with checksum length (4 bytes)  |
+  // | checksum (4 bytes)                     |
+  //
+  // Fields description:
+  // - mapId: Unique identifier for the map (4 bytes)
+  // - attemptId: Identifier for the attempt (4 bytes)
+  // - batchId with checksum flag:
+  //   -- checksum flag: 1 bit (indicates if batchId has a checksum)
+  //   -- batchId: 31 bits (always positive when represented as an integer)
+  // - length with checksum length: total length of the data + 4 bytes for 
checksum
+  // - checksum: Always positive integer (4 bytes)
+  public static final int BATCH_HEADER_SIZE = 5 * 4;
+  public static final int BATCH_HEADER_SIZE_WITHOUT_CHECKSUM = 
BATCH_HEADER_SIZE - 4;
+  public static final int MAP_ID_OFFSET = Platform.BYTE_ARRAY_OFFSET;
+  public static final int ATTEMPT_ID_OFFSET = Platform.BYTE_ARRAY_OFFSET + 4;
+  public static final int BATCH_ID_OFFSET = Platform.BYTE_ARRAY_OFFSET + 8;
+  public static final int LENGTH_OFFSET = Platform.BYTE_ARRAY_OFFSET + 12;
+  public static final int CHECKSUM_OFFSET = Platform.BYTE_ARRAY_OFFSET + 16;
+  public static final int POSITIVE_MASK = 0x7FFFFFFF;
+  public static final int HIGHEST_1_BIT_FLAG_MASK = 0x80000000;

Review Comment:
   I agree with this design that the batchId can not be a negative number. But 
this class needs to be compatible with the old clients. During cluster upgrade, 
there will be a moment when old clients are talking to new servers.



##########
client-flink/common/src/main/java/org/apache/celeborn/plugin/flink/readclient/FlinkShuffleClientImpl.java:
##########
@@ -79,6 +76,7 @@ public class FlinkShuffleClientImpl extends ShuffleClientImpl 
{
   private ConcurrentHashMap<String, TransportClient> currentClient =
       JavaUtils.newConcurrentHashMap();
   private long driverTimestamp;
+  private final int BATCH_HEADER_SIZE = 4 * 4;

Review Comment:
   Ditto.



##########
common/src/main/java/org/apache/celeborn/common/util/PushDataHeaderUtils.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.common.util;
+
+import java.util.zip.CRC32;
+
+import org.apache.celeborn.common.unsafe.Platform;
+
+public class PushDataHeaderUtils {
+  // Data Header Layout:
+  // | mapId (4 bytes)                        |
+  // | attemptId (4 bytes)                    |
+  // | batchId with checksum flag (4 bytes)   |
+  // | length with checksum length (4 bytes)  |
+  // | checksum (4 bytes)                     |
+  //
+  // Fields description:
+  // - mapId: Unique identifier for the map (4 bytes)
+  // - attemptId: Identifier for the attempt (4 bytes)
+  // - batchId with checksum flag:
+  //   -- checksum flag: 1 bit (indicates if batchId has a checksum)
+  //   -- batchId: 31 bits (always positive when represented as an integer)
+  // - length with checksum length: total length of the data + 4 bytes for 
checksum
+  // - checksum: Always positive integer (4 bytes)
+  public static final int BATCH_HEADER_SIZE = 5 * 4;
+  public static final int BATCH_HEADER_SIZE_WITHOUT_CHECKSUM = 
BATCH_HEADER_SIZE - 4;
+  public static final int MAP_ID_OFFSET = Platform.BYTE_ARRAY_OFFSET;
+  public static final int ATTEMPT_ID_OFFSET = Platform.BYTE_ARRAY_OFFSET + 4;
+  public static final int BATCH_ID_OFFSET = Platform.BYTE_ARRAY_OFFSET + 8;
+  public static final int LENGTH_OFFSET = Platform.BYTE_ARRAY_OFFSET + 12;
+  public static final int CHECKSUM_OFFSET = Platform.BYTE_ARRAY_OFFSET + 16;
+  public static final int POSITIVE_MASK = 0x7FFFFFFF;
+  public static final int HIGHEST_1_BIT_FLAG_MASK = 0x80000000;
+
+  public static void buildDataHeader(
+      byte[] data, int mapId, int attemptId, int batchId, int length) {
+    assert data.length >= BATCH_HEADER_SIZE;
+    int batchIdWithChecksumFlag = batchIdWithChecksumFlag(batchId);
+    int lengthWithChecksum = length + 4;
+    Platform.putInt(data, MAP_ID_OFFSET, mapId);
+    Platform.putInt(data, ATTEMPT_ID_OFFSET, attemptId);
+    Platform.putInt(data, BATCH_ID_OFFSET, batchIdWithChecksumFlag);
+    Platform.putInt(data, LENGTH_OFFSET, lengthWithChecksum);
+    Platform.putInt(data, CHECKSUM_OFFSET, computeHeaderChecksum32(data));
+  }
+
+  public static int batchIdWithChecksumFlag(int batchId) {
+    return batchId | HIGHEST_1_BIT_FLAG_MASK;
+  }
+
+  public static int batchIdWithoutChecksumFlag(int batchId) {
+    return batchId & POSITIVE_MASK;
+  }
+
+  public static boolean hasChecksumFlag(byte[] data) {
+    int batchId = Platform.getInt(data, BATCH_ID_OFFSET);
+    return (batchId & HIGHEST_1_BIT_FLAG_MASK) != 0;
+  }
+
+  public static int getMapId(byte[] data) {
+    return Platform.getInt(data, MAP_ID_OFFSET);
+  }
+
+  public static int getAttemptId(byte[] data) {
+    return Platform.getInt(data, ATTEMPT_ID_OFFSET);
+  }
+
+  public static int getBatchId(byte[] data) {
+    return batchIdWithoutChecksumFlag(Platform.getInt(data, BATCH_ID_OFFSET));
+  }
+
+  public static int getLength(byte[] data) {
+    return Platform.getInt(data, LENGTH_OFFSET) - 4;
+  }
+
+  public static int computeHeaderChecksum32(byte[] data) {
+    assert data.length >= BATCH_HEADER_SIZE_WITHOUT_CHECKSUM;
+    CRC32 crc32 = new CRC32();
+    crc32.update(data, 0, BATCH_HEADER_SIZE_WITHOUT_CHECKSUM);

Review Comment:
   Although 16-byte calculation for CRC can be trivial, there can be enormous 
push data structures to handle for a Celeborn Worker. I think it would be 
better to add a switch to let the users enable or disable this feature. For 
some users, the never meet any issues about this, they can just disable this 
feature to save the CPU. 



##########
common/src/main/java/org/apache/celeborn/common/util/PushDataHeaderUtils.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.common.util;
+
+import java.util.zip.CRC32;
+
+import org.apache.celeborn.common.unsafe.Platform;
+
+public class PushDataHeaderUtils {
+  // Data Header Layout:
+  // | mapId (4 bytes)                        |
+  // | attemptId (4 bytes)                    |
+  // | batchId with checksum flag (4 bytes)   |
+  // | length with checksum length (4 bytes)  |
+  // | checksum (4 bytes)                     |
+  //
+  // Fields description:
+  // - mapId: Unique identifier for the map (4 bytes)
+  // - attemptId: Identifier for the attempt (4 bytes)
+  // - batchId with checksum flag:
+  //   -- checksum flag: 1 bit (indicates if batchId has a checksum)
+  //   -- batchId: 31 bits (always positive when represented as an integer)
+  // - length with checksum length: total length of the data + 4 bytes for 
checksum
+  // - checksum: Always positive integer (4 bytes)
+  public static final int BATCH_HEADER_SIZE = 5 * 4;
+  public static final int BATCH_HEADER_SIZE_WITHOUT_CHECKSUM = 
BATCH_HEADER_SIZE - 4;
+  public static final int MAP_ID_OFFSET = Platform.BYTE_ARRAY_OFFSET;
+  public static final int ATTEMPT_ID_OFFSET = Platform.BYTE_ARRAY_OFFSET + 4;
+  public static final int BATCH_ID_OFFSET = Platform.BYTE_ARRAY_OFFSET + 8;
+  public static final int LENGTH_OFFSET = Platform.BYTE_ARRAY_OFFSET + 12;
+  public static final int CHECKSUM_OFFSET = Platform.BYTE_ARRAY_OFFSET + 16;
+  public static final int POSITIVE_MASK = 0x7FFFFFFF;
+  public static final int HIGHEST_1_BIT_FLAG_MASK = 0x80000000;
+
+  public static void buildDataHeader(
+      byte[] data, int mapId, int attemptId, int batchId, int length) {
+    assert data.length >= BATCH_HEADER_SIZE;
+    int batchIdWithChecksumFlag = batchIdWithChecksumFlag(batchId);
+    int lengthWithChecksum = length + 4;
+    Platform.putInt(data, MAP_ID_OFFSET, mapId);
+    Platform.putInt(data, ATTEMPT_ID_OFFSET, attemptId);
+    Platform.putInt(data, BATCH_ID_OFFSET, batchIdWithChecksumFlag);
+    Platform.putInt(data, LENGTH_OFFSET, lengthWithChecksum);
+    Platform.putInt(data, CHECKSUM_OFFSET, computeHeaderChecksum32(data));
+  }
+
+  public static int batchIdWithChecksumFlag(int batchId) {
+    return batchId | HIGHEST_1_BIT_FLAG_MASK;
+  }
+
+  public static int batchIdWithoutChecksumFlag(int batchId) {
+    return batchId & POSITIVE_MASK;
+  }
+
+  public static boolean hasChecksumFlag(byte[] data) {
+    int batchId = Platform.getInt(data, BATCH_ID_OFFSET);
+    return (batchId & HIGHEST_1_BIT_FLAG_MASK) != 0;
+  }
+
+  public static int getMapId(byte[] data) {
+    return Platform.getInt(data, MAP_ID_OFFSET);
+  }
+
+  public static int getAttemptId(byte[] data) {
+    return Platform.getInt(data, ATTEMPT_ID_OFFSET);
+  }
+
+  public static int getBatchId(byte[] data) {
+    return batchIdWithoutChecksumFlag(Platform.getInt(data, BATCH_ID_OFFSET));
+  }
+
+  public static int getLength(byte[] data) {
+    return Platform.getInt(data, LENGTH_OFFSET) - 4;
+  }
+
+  public static int computeHeaderChecksum32(byte[] data) {
+    assert data.length >= BATCH_HEADER_SIZE_WITHOUT_CHECKSUM;
+    CRC32 crc32 = new CRC32();
+    crc32.update(data, 0, BATCH_HEADER_SIZE_WITHOUT_CHECKSUM);

Review Comment:
   I have received some reports from users telling me that Celeborn workers are 
consuming too much CPU during rush hours. If this feature is on by default, 
this will surely get things worse.
   You can make this config a client-side control config and pass it in the 
ReserveSlots request.



-- 
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]

Reply via email to