zwangsheng commented on code in PR #2979: URL: https://github.com/apache/celeborn/pull/2979#discussion_r1875588404
########## worker/src/test/scala/org/apache/celeborn/service/deploy/cluster/PushDataWithChecksumSuite.scala: ########## @@ -0,0 +1,120 @@ +/* + * 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.service.deploy.cluster + +import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets + +import io.netty.buffer.Unpooled +import org.apache.commons.lang3.RandomStringUtils +import org.scalatest.BeforeAndAfterAll +import org.scalatest.funsuite.AnyFunSuite + +import org.apache.celeborn.client.{LifecycleManager, ShuffleClientImpl} +import org.apache.celeborn.common.CelebornConf +import org.apache.celeborn.common.identity.UserIdentifier +import org.apache.celeborn.common.internal.Logging +import org.apache.celeborn.common.network.buffer.NettyManagedBuffer +import org.apache.celeborn.common.network.client.RpcResponseCallback +import org.apache.celeborn.common.network.protocol.PushData +import org.apache.celeborn.common.protocol.PartitionLocation +import org.apache.celeborn.common.protocol.message.StatusCode +import org.apache.celeborn.common.unsafe.Platform +import org.apache.celeborn.common.util.{PushDataHeaderUtils, Utils} +import org.apache.celeborn.service.deploy.MiniClusterFeature + +class PushDataWithChecksumSuite extends AnyFunSuite + with Logging with MiniClusterFeature with BeforeAndAfterAll { + var masterPort = 19097 + + override def beforeAll(): Unit = { + logInfo("test initialized , setup Celeborn mini cluster") + val (master, _) = setupMiniClusterWithRandomPorts() + masterPort = master.conf.masterPort + } + + override def afterAll(): Unit = { + logInfo("all test complete , stop Celeborn mini cluster") + shutdownMiniCluster() + } + + test("push data and checksum error") { + val APP = s"app-${System.currentTimeMillis()}" + val SHUFFLE_ID = 0 + val MAP_ID = 0 + val ATTEMPT_ID = 0 + val MAP_NUM = 1 + val PARTITION_NUM = 3 + + val clientConf = new CelebornConf() + .set(CelebornConf.MASTER_ENDPOINTS.key, s"localhost:$masterPort") + val lifecycleManager = new LifecycleManager(APP, clientConf) + val shuffleClient = new ShuffleClientImpl(APP, clientConf, UserIdentifier("mock", "mock")) + shuffleClient.setupLifecycleManagerRef(lifecycleManager.self) + + // ping and reserveSlots + val DATA0 = RandomStringUtils.secure().next(10).getBytes(StandardCharsets.UTF_8) + shuffleClient.pushData( + SHUFFLE_ID, + MAP_ID, + ATTEMPT_ID, + 0, + DATA0, + 0, + DATA0.length, + MAP_NUM, + PARTITION_NUM) + + val partitionLocationMap = + shuffleClient.getPartitionLocation(SHUFFLE_ID, MAP_NUM, PARTITION_NUM) + val location = partitionLocationMap.get(SHUFFLE_ID) + val shuffleKey = Utils.makeShuffleKey(APP, SHUFFLE_ID) + + val batchId = 123 // mock batch id + val bodyLength = 100 + val buffer: Array[Byte] = Array.fill(PushDataHeaderUtils.BATCH_HEADER_SIZE)( + 0.toByte) ++ RandomStringUtils.secure().next(bodyLength).getBytes(StandardCharsets.UTF_8) + PushDataHeaderUtils.buildDataHeader(buffer, MAP_ID, ATTEMPT_ID, batchId, bodyLength) + val checksum = PushDataHeaderUtils.computeHeaderChecksum32(buffer) + Platform.putInt(buffer, PushDataHeaderUtils.CHECKSUM_OFFSET, checksum + 1) // error checksum + + val nettyBuffer = new NettyManagedBuffer(Unpooled.wrappedBuffer(buffer)); + val client = + shuffleClient.getDataClientFactory.createClient(location.getHost, location.getPushPort) + val pushData = new PushData( + PartitionLocation.Mode.PRIMARY.mode, + shuffleKey, + location.getUniqueId, + nettyBuffer) + val callback = new RpcResponseCallback() { + override def onSuccess(response: ByteBuffer): Unit = { + assert(false, "push data should fail") Review Comment: using scalatest fail instead ########## 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: +1, we should think about how maintain compatibility with older versions. ########## worker/src/test/scala/org/apache/celeborn/service/deploy/cluster/PushDataWithChecksumSuite.scala: ########## @@ -0,0 +1,120 @@ +/* + * 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.service.deploy.cluster + +import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets + +import io.netty.buffer.Unpooled +import org.apache.commons.lang3.RandomStringUtils +import org.scalatest.BeforeAndAfterAll +import org.scalatest.funsuite.AnyFunSuite + +import org.apache.celeborn.client.{LifecycleManager, ShuffleClientImpl} +import org.apache.celeborn.common.CelebornConf +import org.apache.celeborn.common.identity.UserIdentifier +import org.apache.celeborn.common.internal.Logging +import org.apache.celeborn.common.network.buffer.NettyManagedBuffer +import org.apache.celeborn.common.network.client.RpcResponseCallback +import org.apache.celeborn.common.network.protocol.PushData +import org.apache.celeborn.common.protocol.PartitionLocation +import org.apache.celeborn.common.protocol.message.StatusCode +import org.apache.celeborn.common.unsafe.Platform +import org.apache.celeborn.common.util.{PushDataHeaderUtils, Utils} +import org.apache.celeborn.service.deploy.MiniClusterFeature + +class PushDataWithChecksumSuite extends AnyFunSuite + with Logging with MiniClusterFeature with BeforeAndAfterAll { + var masterPort = 19097 + + override def beforeAll(): Unit = { + logInfo("test initialized , setup Celeborn mini cluster") + val (master, _) = setupMiniClusterWithRandomPorts() + masterPort = master.conf.masterPort + } + + override def afterAll(): Unit = { + logInfo("all test complete , stop Celeborn mini cluster") + shutdownMiniCluster() + } + + test("push data and checksum error") { + val APP = s"app-${System.currentTimeMillis()}" + val SHUFFLE_ID = 0 + val MAP_ID = 0 + val ATTEMPT_ID = 0 + val MAP_NUM = 1 + val PARTITION_NUM = 3 + + val clientConf = new CelebornConf() + .set(CelebornConf.MASTER_ENDPOINTS.key, s"localhost:$masterPort") + val lifecycleManager = new LifecycleManager(APP, clientConf) + val shuffleClient = new ShuffleClientImpl(APP, clientConf, UserIdentifier("mock", "mock")) + shuffleClient.setupLifecycleManagerRef(lifecycleManager.self) + + // ping and reserveSlots + val DATA0 = RandomStringUtils.secure().next(10).getBytes(StandardCharsets.UTF_8) + shuffleClient.pushData( + SHUFFLE_ID, + MAP_ID, + ATTEMPT_ID, + 0, + DATA0, + 0, + DATA0.length, + MAP_NUM, + PARTITION_NUM) + + val partitionLocationMap = + shuffleClient.getPartitionLocation(SHUFFLE_ID, MAP_NUM, PARTITION_NUM) + val location = partitionLocationMap.get(SHUFFLE_ID) + val shuffleKey = Utils.makeShuffleKey(APP, SHUFFLE_ID) + + val batchId = 123 // mock batch id + val bodyLength = 100 + val buffer: Array[Byte] = Array.fill(PushDataHeaderUtils.BATCH_HEADER_SIZE)( + 0.toByte) ++ RandomStringUtils.secure().next(bodyLength).getBytes(StandardCharsets.UTF_8) + PushDataHeaderUtils.buildDataHeader(buffer, MAP_ID, ATTEMPT_ID, batchId, bodyLength) + val checksum = PushDataHeaderUtils.computeHeaderChecksum32(buffer) + Platform.putInt(buffer, PushDataHeaderUtils.CHECKSUM_OFFSET, checksum + 1) // error checksum + + val nettyBuffer = new NettyManagedBuffer(Unpooled.wrappedBuffer(buffer)); + val client = + shuffleClient.getDataClientFactory.createClient(location.getHost, location.getPushPort) + val pushData = new PushData( + PartitionLocation.Mode.PRIMARY.mode, + shuffleKey, + location.getUniqueId, + nettyBuffer) + val callback = new RpcResponseCallback() { + override def onSuccess(response: ByteBuffer): Unit = { + assert(false, "push data should fail") + } + + override def onFailure(e: Throwable): Unit = { + if (e.getMessage == StatusCode.PUSH_DATA_CHECKSUM_FAIL.toString) { Review Comment: assert(e.getMessage == StatusCode.PUSH_DATA_CHECKSUM_FAIL.toString, "push data should fail for checksum error, but get " + e.getMessage) -- 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]
