From 1425a7a518663d855f2cf21497547f983a60315b Mon Sep 17 00:00:00 2001 From: wankunde Date: Sun, 21 Jul 2024 23:20:21 +0800 Subject: [PATCH 1/3] Use ShuffleSegment to replace FileBasedShuffleSegment and BufferSegment --- .../client/impl/ShuffleReadClientImpl.java | 12 +- .../org/apache/uniffle/client/TestUtils.java | 12 +- .../uniffle/common/ShuffleDataResult.java | 28 ++-- .../uniffle/common/ShuffleDataSegment.java | 10 +- ...BufferSegment.java => ShuffleSegment.java} | 33 +++-- .../common/netty/protocol/Decoders.java | 14 +- .../common/netty/protocol/Encoders.java | 24 ++-- .../GetMemoryShuffleDataResponse.java | 34 ++--- .../segment/FixedSizeSegmentSplitter.java | 14 +- .../segment/LocalOrderSegmentSplitter.java | 16 +-- .../uniffle/common/ShuffleDataResultTest.java | 2 +- ...gmentTest.java => ShuffleSegmentTest.java} | 18 +-- .../netty/TransportFrameDecoderTest.java | 8 +- .../netty/protocol/NettyProtocolTest.java | 8 +- .../LocalOrderSegmentSplitterTest.java | 76 +++++------ .../test/ShuffleServerFaultToleranceTest.java | 8 +- ...huffleServerWithLocalOfLocalOrderTest.java | 6 +- .../test/ShuffleServerWithLocalTest.java | 6 +- .../ShuffleServerWithMemLocalHadoopTest.java | 12 +- .../test/ShuffleServerWithMemoryTest.java | 12 +- .../impl/grpc/ShuffleServerGrpcClient.java | 8 +- .../RssGetInMemoryShuffleDataResponse.java | 16 +-- .../server/ShuffleServerGrpcService.java | 14 +- .../server/buffer/AbstractShuffleBuffer.java | 24 ++-- .../buffer/ShuffleBufferWithLinkedList.java | 24 ++-- .../buffer/ShuffleBufferWithSkipList.java | 20 +-- .../netty/ShuffleServerNettyHandler.java | 8 +- .../server/ShuffleFlushManagerTest.java | 8 +- .../server/ShuffleTaskManagerTest.java | 6 +- .../ShuffleBufferWithLinkedListTest.java | 12 +- .../buffer/ShuffleBufferWithSkipListTest.java | 6 +- .../uniffle/storage/api/FileWriter.java | 4 +- .../common/FileBasedShuffleSegment.java | 128 ------------------ .../storage/common/ShuffleSegment.java | 20 --- .../handler/api/ClientReadHandler.java | 4 +- .../impl/AbstractClientReadHandler.java | 6 +- .../impl/ComposedClientReadHandler.java | 4 +- .../storage/handler/impl/DataFileSegment.java | 14 +- .../handler/impl/HadoopFileWriter.java | 5 +- .../impl/HadoopShuffleReadHandler.java | 6 +- .../impl/HadoopShuffleWriteHandler.java | 6 +- .../impl/LocalFileServerReadHandler.java | 6 +- .../handler/impl/LocalFileWriteHandler.java | 6 +- .../storage/handler/impl/LocalFileWriter.java | 4 +- .../handler/impl/MemoryClientReadHandler.java | 6 +- .../impl/MultiReplicaClientReadHandler.java | 4 +- .../storage/util/ShuffleStorageUtils.java | 31 ++--- .../storage/HadoopShuffleHandlerTestBase.java | 15 +- .../impl/HadoopClientReadHandlerTest.java | 9 +- .../handler/impl/HadoopFileReaderTest.java | 8 +- .../handler/impl/HadoopFileWriterTest.java | 4 +- .../handler/impl/HadoopHandlerTest.java | 13 +- .../impl/HadoopShuffleReadHandlerTest.java | 15 +- .../impl/LocalFileHandlerTestBase.java | 13 +- .../impl/LocalFileServerReadHandlerTest.java | 6 +- .../storage/util/ShuffleStorageUtilsTest.java | 119 ++++++++-------- 56 files changed, 408 insertions(+), 547 deletions(-) rename common/src/main/java/org/apache/uniffle/common/{BufferSegment.java => ShuffleSegment.java} (72%) rename common/src/test/java/org/apache/uniffle/common/{BufferSegmentTest.java => ShuffleSegmentTest.java} (77%) delete mode 100644 storage/src/main/java/org/apache/uniffle/storage/common/FileBasedShuffleSegment.java delete mode 100644 storage/src/main/java/org/apache/uniffle/storage/common/ShuffleSegment.java diff --git a/client/src/main/java/org/apache/uniffle/client/impl/ShuffleReadClientImpl.java b/client/src/main/java/org/apache/uniffle/client/impl/ShuffleReadClientImpl.java index e1aa0f9582..02bc1a7e58 100644 --- a/client/src/main/java/org/apache/uniffle/client/impl/ShuffleReadClientImpl.java +++ b/client/src/main/java/org/apache/uniffle/client/impl/ShuffleReadClientImpl.java @@ -35,7 +35,7 @@ import org.apache.uniffle.client.factory.ShuffleClientFactory; import org.apache.uniffle.client.response.CompressedShuffleBlock; import org.apache.uniffle.client.util.DefaultIdHelper; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataDistributionType; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShuffleServerInfo; @@ -62,7 +62,7 @@ public class ShuffleReadClientImpl implements ShuffleReadClient { private Roaring64NavigableMap taskIdBitmap; private Roaring64NavigableMap pendingBlockIds; private Roaring64NavigableMap processedBlockIds = Roaring64NavigableMap.bitmapOf(); - private Queue bufferSegmentQueue = Queues.newLinkedBlockingQueue(); + private Queue shuffleSegmentQueue = Queues.newLinkedBlockingQueue(); private AtomicLong readDataTime = new AtomicLong(0); private AtomicLong copyTime = new AtomicLong(0); private AtomicLong crcCheckTime = new AtomicLong(0); @@ -212,19 +212,19 @@ public CompressedShuffleBlock readShuffleBlockData() { } // if client need request new data from shuffle server - if (bufferSegmentQueue.isEmpty()) { + if (shuffleSegmentQueue.isEmpty()) { if (read() <= 0) { return null; } } // get next buffer segment - BufferSegment bs = null; + ShuffleSegment bs = null; // blocks in bufferSegmentQueue may be from different partition in range partition mode, // or may be from speculation task, filter them and just read the necessary block while (true) { - bs = bufferSegmentQueue.poll(); + bs = shuffleSegmentQueue.poll(); if (bs == null) { break; } @@ -317,7 +317,7 @@ private int read() { if (readBuffer == null || readBuffer.capacity() == 0) { return 0; } - bufferSegmentQueue.addAll(sdr.getBufferSegments()); + shuffleSegmentQueue.addAll(sdr.getBufferSegments()); return sdr.getBufferSegments().size(); } diff --git a/client/src/test/java/org/apache/uniffle/client/TestUtils.java b/client/src/test/java/org/apache/uniffle/client/TestUtils.java index 8b284567ee..1f54fea816 100644 --- a/client/src/test/java/org/apache/uniffle/client/TestUtils.java +++ b/client/src/test/java/org/apache/uniffle/client/TestUtils.java @@ -24,7 +24,7 @@ import org.apache.uniffle.client.api.ShuffleReadClient; import org.apache.uniffle.client.response.CompressedShuffleBlock; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; import static org.junit.jupiter.api.Assertions.assertArrayEquals; @@ -65,10 +65,10 @@ public static void validateResult(ShuffleReadClient readClient, Map expectedData, ShuffleDataResult sdr) { byte[] buffer = sdr.getData(); - List bufferSegments = sdr.getBufferSegments(); - assertEquals(expectedData.size(), bufferSegments.size()); + List shuffleSegments = sdr.getBufferSegments(); + assertEquals(expectedData.size(), shuffleSegments.size()); for (Map.Entry entry : expectedData.entrySet()) { - BufferSegment bs = findBufferSegment(entry.getKey(), bufferSegments); + ShuffleSegment bs = findBufferSegment(entry.getKey(), shuffleSegments); assertNotNull(bs); byte[] data = new byte[bs.getLength()]; System.arraycopy(buffer, bs.getOffset(), data, 0, bs.getLength()); @@ -76,8 +76,8 @@ public static void validateResult(Map expectedData, ShuffleDataRes } } - private static BufferSegment findBufferSegment(long blockId, List bufferSegments) { - for (BufferSegment bs : bufferSegments) { + private static ShuffleSegment findBufferSegment(long blockId, List shuffleSegments) { + for (ShuffleSegment bs : shuffleSegments) { if (bs.getBlockId() == blockId) { return bs; } diff --git a/common/src/main/java/org/apache/uniffle/common/ShuffleDataResult.java b/common/src/main/java/org/apache/uniffle/common/ShuffleDataResult.java index 00867e7006..3c437aec12 100644 --- a/common/src/main/java/org/apache/uniffle/common/ShuffleDataResult.java +++ b/common/src/main/java/org/apache/uniffle/common/ShuffleDataResult.java @@ -31,7 +31,7 @@ public class ShuffleDataResult { private final ManagedBuffer buffer; - private final List bufferSegments; + private final List shuffleSegments; public ShuffleDataResult() { this(new byte[0]); @@ -43,26 +43,26 @@ public ShuffleDataResult(byte[] data) { public ShuffleDataResult(ManagedBuffer buffer) { this.buffer = buffer; - this.bufferSegments = Lists.newArrayList(); + this.shuffleSegments = Lists.newArrayList(); } - public ShuffleDataResult(ByteBuffer data, List bufferSegments) { + public ShuffleDataResult(ByteBuffer data, List shuffleSegments) { this.buffer = new NettyManagedBuffer(data != null ? Unpooled.wrappedBuffer(data) : Unpooled.EMPTY_BUFFER); - this.bufferSegments = bufferSegments; + this.shuffleSegments = shuffleSegments; } - public ShuffleDataResult(ByteBuf data, List bufferSegments) { - this(new NettyManagedBuffer(data), bufferSegments); + public ShuffleDataResult(ByteBuf data, List shuffleSegments) { + this(new NettyManagedBuffer(data), shuffleSegments); } - public ShuffleDataResult(byte[] data, List bufferSegments) { - this(data != null ? ByteBuffer.wrap(data) : null, bufferSegments); + public ShuffleDataResult(byte[] data, List shuffleSegments) { + this(data != null ? ByteBuffer.wrap(data) : null, shuffleSegments); } - public ShuffleDataResult(ManagedBuffer data, List bufferSegments) { + public ShuffleDataResult(ManagedBuffer data, List shuffleSegments) { this.buffer = data; - this.bufferSegments = bufferSegments; + this.shuffleSegments = shuffleSegments; } public byte[] getData() { @@ -94,13 +94,13 @@ public ManagedBuffer getManagedBuffer() { return buffer; } - public List getBufferSegments() { - return bufferSegments; + public List getBufferSegments() { + return shuffleSegments; } public boolean isEmpty() { - return bufferSegments == null - || bufferSegments.isEmpty() + return shuffleSegments == null + || shuffleSegments.isEmpty() || buffer == null || buffer.size() == 0; } diff --git a/common/src/main/java/org/apache/uniffle/common/ShuffleDataSegment.java b/common/src/main/java/org/apache/uniffle/common/ShuffleDataSegment.java index af7299087c..02d919d9f4 100644 --- a/common/src/main/java/org/apache/uniffle/common/ShuffleDataSegment.java +++ b/common/src/main/java/org/apache/uniffle/common/ShuffleDataSegment.java @@ -27,12 +27,12 @@ public class ShuffleDataSegment { private final long offset; private final int length; - private final List bufferSegments; + private final List shuffleSegments; - public ShuffleDataSegment(long offset, int length, List bufferSegments) { + public ShuffleDataSegment(long offset, int length, List shuffleSegments) { this.offset = offset; this.length = length; - this.bufferSegments = bufferSegments; + this.shuffleSegments = shuffleSegments; } public long getOffset() { @@ -43,7 +43,7 @@ public int getLength() { return length; } - public List getBufferSegments() { - return bufferSegments; + public List getBufferSegments() { + return shuffleSegments; } } diff --git a/common/src/main/java/org/apache/uniffle/common/BufferSegment.java b/common/src/main/java/org/apache/uniffle/common/ShuffleSegment.java similarity index 72% rename from common/src/main/java/org/apache/uniffle/common/BufferSegment.java rename to common/src/main/java/org/apache/uniffle/common/ShuffleSegment.java index 1ab68c5121..4b62dce188 100644 --- a/common/src/main/java/org/apache/uniffle/common/BufferSegment.java +++ b/common/src/main/java/org/apache/uniffle/common/ShuffleSegment.java @@ -21,7 +21,12 @@ import org.apache.uniffle.common.exception.RssException; -public class BufferSegment { +/** + * The metadata for a segment, it can be a memory segment or a file segment. + */ +public class ShuffleSegment implements java.io.Serializable, Comparable { + + public static final int SEGMENT_SIZE = 4 * Long.BYTES + 2 * Integer.BYTES; private long blockId; private long offset; @@ -30,7 +35,7 @@ public class BufferSegment { private long crc; private long taskAttemptId; - public BufferSegment( + public ShuffleSegment( long blockId, long offset, int length, int uncompressLength, long crc, long taskAttemptId) { this.blockId = blockId; this.offset = offset; @@ -42,13 +47,13 @@ public BufferSegment( @Override public boolean equals(Object obj) { - if (obj instanceof BufferSegment) { - return blockId == ((BufferSegment) obj).getBlockId() - && offset == ((BufferSegment) obj).getOffset() - && length == ((BufferSegment) obj).getLength() - && uncompressLength == ((BufferSegment) obj).getUncompressLength() - && crc == ((BufferSegment) obj).getCrc() - && taskAttemptId == ((BufferSegment) obj).getTaskAttemptId(); + if (obj instanceof ShuffleSegment) { + return blockId == ((ShuffleSegment) obj).getBlockId() + && offset == ((ShuffleSegment) obj).getOffset() + && length == ((ShuffleSegment) obj).getLength() + && uncompressLength == ((ShuffleSegment) obj).getUncompressLength() + && crc == ((ShuffleSegment) obj).getCrc() + && taskAttemptId == ((ShuffleSegment) obj).getTaskAttemptId(); } return false; } @@ -101,4 +106,14 @@ public int getUncompressLength() { public long getTaskAttemptId() { return taskAttemptId; } + + @Override + public int compareTo(ShuffleSegment s) { + if (this.offset > s.getOffset()) { + return 1; + } else if (this.offset < s.getOffset()) { + return -1; + } + return 0; + } } 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 fc70048808..0d64ba2ba6 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 @@ -24,7 +24,7 @@ import com.google.common.collect.Maps; import io.netty.buffer.ByteBuf; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleBlockInfo; import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.util.ByteBufUtils; @@ -84,8 +84,8 @@ public static Map> decodePartitionToBlockIds(ByteBuf byteBuf return partitionToBlockIds; } - public static List decodeBufferSegments(ByteBuf byteBuf) { - List bufferSegments = Lists.newArrayList(); + public static List decodeBufferSegments(ByteBuf byteBuf) { + List shuffleSegments = Lists.newArrayList(); int size = byteBuf.readInt(); for (int i = 0; i < size; i++) { long blockId = byteBuf.readLong(); @@ -94,10 +94,10 @@ public static List decodeBufferSegments(ByteBuf byteBuf) { int uncompressLength = byteBuf.readInt(); long crc = byteBuf.readLong(); long taskAttemptId = byteBuf.readLong(); - BufferSegment bufferSegment = - new BufferSegment(blockId, offset, length, uncompressLength, crc, taskAttemptId); - bufferSegments.add(bufferSegment); + ShuffleSegment shuffleSegment = + new ShuffleSegment(blockId, offset, length, uncompressLength, crc, taskAttemptId); + shuffleSegments.add(shuffleSegment); } - return bufferSegments; + return shuffleSegments; } } diff --git a/common/src/main/java/org/apache/uniffle/common/netty/protocol/Encoders.java b/common/src/main/java/org/apache/uniffle/common/netty/protocol/Encoders.java index b74a517b90..54ea36b729 100644 --- a/common/src/main/java/org/apache/uniffle/common/netty/protocol/Encoders.java +++ b/common/src/main/java/org/apache/uniffle/common/netty/protocol/Encoders.java @@ -21,7 +21,7 @@ import io.netty.buffer.ByteBuf; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.ShuffleBlockInfo; import org.apache.uniffle.common.ShuffleServerInfo; @@ -80,19 +80,19 @@ public static void encodePartitionRanges(List partitionRanges, B } } - public static void encodeBufferSegments(List bufferSegments, ByteBuf byteBuf) { - byteBuf.writeInt(bufferSegments.size()); - for (BufferSegment bufferSegment : bufferSegments) { - byteBuf.writeLong(bufferSegment.getBlockId()); - byteBuf.writeInt(bufferSegment.getOffset()); - byteBuf.writeInt(bufferSegment.getLength()); - byteBuf.writeInt(bufferSegment.getUncompressLength()); - byteBuf.writeLong(bufferSegment.getCrc()); - byteBuf.writeLong(bufferSegment.getTaskAttemptId()); + public static void encodeBufferSegments(List shuffleSegments, ByteBuf byteBuf) { + byteBuf.writeInt(shuffleSegments.size()); + for (ShuffleSegment shuffleSegment : shuffleSegments) { + byteBuf.writeLong(shuffleSegment.getBlockId()); + byteBuf.writeInt(shuffleSegment.getOffset()); + byteBuf.writeInt(shuffleSegment.getLength()); + byteBuf.writeInt(shuffleSegment.getUncompressLength()); + byteBuf.writeLong(shuffleSegment.getCrc()); + byteBuf.writeLong(shuffleSegment.getTaskAttemptId()); } } - public static int encodeLengthOfBufferSegments(List bufferSegments) { - return Integer.BYTES + bufferSegments.size() * (3 * Long.BYTES + 3 * Integer.BYTES); + public static int encodeLengthOfBufferSegments(List shuffleSegments) { + return Integer.BYTES + shuffleSegments.size() * (3 * Long.BYTES + 3 * Integer.BYTES); } } diff --git a/common/src/main/java/org/apache/uniffle/common/netty/protocol/GetMemoryShuffleDataResponse.java b/common/src/main/java/org/apache/uniffle/common/netty/protocol/GetMemoryShuffleDataResponse.java index c9b9f7ed5c..3c79b1d277 100644 --- a/common/src/main/java/org/apache/uniffle/common/netty/protocol/GetMemoryShuffleDataResponse.java +++ b/common/src/main/java/org/apache/uniffle/common/netty/protocol/GetMemoryShuffleDataResponse.java @@ -22,71 +22,71 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.netty.buffer.ManagedBuffer; import org.apache.uniffle.common.netty.buffer.NettyManagedBuffer; import org.apache.uniffle.common.rpc.StatusCode; import org.apache.uniffle.common.util.ByteBufUtils; public class GetMemoryShuffleDataResponse extends RpcResponse { - private List bufferSegments; + private List shuffleSegments; public GetMemoryShuffleDataResponse( - long requestId, StatusCode statusCode, List bufferSegments, byte[] data) { - this(requestId, statusCode, null, bufferSegments, data); + long requestId, StatusCode statusCode, List shuffleSegments, byte[] data) { + this(requestId, statusCode, null, shuffleSegments, data); } public GetMemoryShuffleDataResponse( long requestId, StatusCode statusCode, String retMessage, - List bufferSegments, + List shuffleSegments, byte[] data) { - this(requestId, statusCode, retMessage, bufferSegments, Unpooled.wrappedBuffer(data)); + this(requestId, statusCode, retMessage, shuffleSegments, Unpooled.wrappedBuffer(data)); } public GetMemoryShuffleDataResponse( long requestId, StatusCode statusCode, String retMessage, - List bufferSegments, + List shuffleSegments, ByteBuf data) { - this(requestId, statusCode, retMessage, bufferSegments, new NettyManagedBuffer(data)); + this(requestId, statusCode, retMessage, shuffleSegments, new NettyManagedBuffer(data)); } public GetMemoryShuffleDataResponse( long requestId, StatusCode statusCode, String retMessage, - List bufferSegments, + List shuffleSegments, ManagedBuffer managedBuffer) { super(requestId, statusCode, retMessage, managedBuffer); - this.bufferSegments = bufferSegments; + this.shuffleSegments = shuffleSegments; } @Override public int encodedLength() { - return super.encodedLength() + Encoders.encodeLengthOfBufferSegments(bufferSegments); + return super.encodedLength() + Encoders.encodeLengthOfBufferSegments(shuffleSegments); } @Override public void encode(ByteBuf buf) { super.encode(buf); - Encoders.encodeBufferSegments(bufferSegments, buf); + Encoders.encodeBufferSegments(shuffleSegments, buf); } public static GetMemoryShuffleDataResponse decode(ByteBuf byteBuf, boolean decodeBody) { long requestId = byteBuf.readLong(); StatusCode statusCode = StatusCode.fromCode(byteBuf.readInt()); String retMessage = ByteBufUtils.readLengthAndString(byteBuf); - List bufferSegments = Decoders.decodeBufferSegments(byteBuf); + List shuffleSegments = Decoders.decodeBufferSegments(byteBuf); if (decodeBody) { NettyManagedBuffer nettyManagedBuffer = new NettyManagedBuffer(byteBuf); return new GetMemoryShuffleDataResponse( - requestId, statusCode, retMessage, bufferSegments, nettyManagedBuffer); + requestId, statusCode, retMessage, shuffleSegments, nettyManagedBuffer); } else { return new GetMemoryShuffleDataResponse( - requestId, statusCode, retMessage, bufferSegments, NettyManagedBuffer.EMPTY_BUFFER); + requestId, statusCode, retMessage, shuffleSegments, NettyManagedBuffer.EMPTY_BUFFER); } } @@ -95,7 +95,7 @@ public Type type() { return Type.GET_MEMORY_SHUFFLE_DATA_RESPONSE; } - public List getBufferSegments() { - return bufferSegments; + public List getBufferSegments() { + return shuffleSegments; } } diff --git a/common/src/main/java/org/apache/uniffle/common/segment/FixedSizeSegmentSplitter.java b/common/src/main/java/org/apache/uniffle/common/segment/FixedSizeSegmentSplitter.java index 04763d8428..b14206b59b 100644 --- a/common/src/main/java/org/apache/uniffle/common/segment/FixedSizeSegmentSplitter.java +++ b/common/src/main/java/org/apache/uniffle/common/segment/FixedSizeSegmentSplitter.java @@ -25,7 +25,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataSegment; import org.apache.uniffle.common.ShuffleIndexResult; import org.apache.uniffle.common.exception.RssException; @@ -52,7 +52,7 @@ public List split(ShuffleIndexResult shuffleIndexResult) { private static List transIndexDataToSegments( ByteBuffer indexData, int readBufferSize, long dataFileLen) { - List bufferSegments = Lists.newArrayList(); + List shuffleSegments = Lists.newArrayList(); List dataFileSegments = Lists.newArrayList(); int bufferOffset = 0; long fileOffset = -1; @@ -91,14 +91,14 @@ private static List transIndexDataToSegments( break; } - bufferSegments.add( - new BufferSegment(blockId, bufferOffset, length, uncompressLength, crc, taskAttemptId)); + shuffleSegments.add( + new ShuffleSegment(blockId, bufferOffset, length, uncompressLength, crc, taskAttemptId)); bufferOffset += length; if (bufferOffset >= readBufferSize) { - ShuffleDataSegment sds = new ShuffleDataSegment(fileOffset, bufferOffset, bufferSegments); + ShuffleDataSegment sds = new ShuffleDataSegment(fileOffset, bufferOffset, shuffleSegments); dataFileSegments.add(sds); - bufferSegments = Lists.newArrayList(); + shuffleSegments = Lists.newArrayList(); bufferOffset = 0; fileOffset = -1; } @@ -108,7 +108,7 @@ private static List transIndexDataToSegments( } if (bufferOffset > 0) { - ShuffleDataSegment sds = new ShuffleDataSegment(fileOffset, bufferOffset, bufferSegments); + ShuffleDataSegment sds = new ShuffleDataSegment(fileOffset, bufferOffset, shuffleSegments); dataFileSegments.add(sds); } diff --git a/common/src/main/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitter.java b/common/src/main/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitter.java index 366968c342..012974b996 100644 --- a/common/src/main/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitter.java +++ b/common/src/main/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitter.java @@ -27,7 +27,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataSegment; import org.apache.uniffle.common.ShuffleIndexResult; import org.apache.uniffle.common.exception.RssException; @@ -66,7 +66,7 @@ public List split(ShuffleIndexResult shuffleIndexResult) { ByteBuffer indexData = shuffleIndexResult.getIndexData(); long dataFileLen = shuffleIndexResult.getDataFileLen(); - List bufferSegments = Lists.newArrayList(); + List shuffleSegments = Lists.newArrayList(); List dataFileSegments = Lists.newArrayList(); int bufferOffset = 0; @@ -115,16 +115,16 @@ public List split(ShuffleIndexResult shuffleIndexResult) { boolean conditionOfDiscontinuousBlocks = lastExpectedBlockIndex != -1 - && bufferSegments.size() > 0 + && shuffleSegments.size() > 0 && expectTaskIds.contains(taskAttemptId) && index - lastExpectedBlockIndex != 1; boolean conditionOfLimitedBufferSize = bufferOffset >= readBufferSize; if (conditionOfDiscontinuousBlocks || conditionOfLimitedBufferSize) { - ShuffleDataSegment sds = new ShuffleDataSegment(fileOffset, bufferOffset, bufferSegments); + ShuffleDataSegment sds = new ShuffleDataSegment(fileOffset, bufferOffset, shuffleSegments); dataFileSegments.add(sds); - bufferSegments = Lists.newArrayList(); + shuffleSegments = Lists.newArrayList(); bufferOffset = 0; fileOffset = -1; } @@ -133,8 +133,8 @@ public List split(ShuffleIndexResult shuffleIndexResult) { if (fileOffset == -1) { fileOffset = offset; } - bufferSegments.add( - new BufferSegment( + shuffleSegments.add( + new ShuffleSegment( blockId, bufferOffset, length, uncompressLength, crc, taskAttemptId)); bufferOffset += length; lastExpectedBlockIndex = index; @@ -146,7 +146,7 @@ public List split(ShuffleIndexResult shuffleIndexResult) { } if (bufferOffset > 0) { - ShuffleDataSegment sds = new ShuffleDataSegment(fileOffset, bufferOffset, bufferSegments); + ShuffleDataSegment sds = new ShuffleDataSegment(fileOffset, bufferOffset, shuffleSegments); dataFileSegments.add(sds); } diff --git a/common/src/test/java/org/apache/uniffle/common/ShuffleDataResultTest.java b/common/src/test/java/org/apache/uniffle/common/ShuffleDataResultTest.java index 5c97eb69b1..3d84271544 100644 --- a/common/src/test/java/org/apache/uniffle/common/ShuffleDataResultTest.java +++ b/common/src/test/java/org/apache/uniffle/common/ShuffleDataResultTest.java @@ -29,7 +29,7 @@ public class ShuffleDataResultTest { @Test public void testEmpty() { - List segments = Collections.singletonList(new BufferSegment(1, 2, 3, 4, 5, 6)); + List segments = Collections.singletonList(new ShuffleSegment(1, 2, 3, 4, 5, 6)); byte[] bytes = null; assertTrue(new ShuffleDataResult().isEmpty()); assertTrue(new ShuffleDataResult(new byte[1]).isEmpty()); diff --git a/common/src/test/java/org/apache/uniffle/common/BufferSegmentTest.java b/common/src/test/java/org/apache/uniffle/common/ShuffleSegmentTest.java similarity index 77% rename from common/src/test/java/org/apache/uniffle/common/BufferSegmentTest.java rename to common/src/test/java/org/apache/uniffle/common/ShuffleSegmentTest.java index ca10485213..a5be4e949b 100644 --- a/common/src/test/java/org/apache/uniffle/common/BufferSegmentTest.java +++ b/common/src/test/java/org/apache/uniffle/common/ShuffleSegmentTest.java @@ -25,12 +25,12 @@ import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertThrows; -public class BufferSegmentTest { +public class ShuffleSegmentTest { @Test public void testEquals() { - BufferSegment segment1 = new BufferSegment(0, 1, 2, 3, 4, 5); - BufferSegment segment2 = new BufferSegment(0, 1, 2, 3, 4, 5); + ShuffleSegment segment1 = new ShuffleSegment(0, 1, 2, 3, 4, 5); + ShuffleSegment segment2 = new ShuffleSegment(0, 1, 2, 3, 4, 5); assertEquals(segment1, segment2); assertEquals(segment1.hashCode(), segment2.hashCode()); assertNotEquals(segment1, null); @@ -48,15 +48,15 @@ public void testEquals() { }) public void testNotEquals( long blockId, long offset, int length, int uncompressLength, long crc, long taskAttemptId) { - BufferSegment segment1 = new BufferSegment(0, 1, 2, 3, 4, 5); - BufferSegment segment2 = - new BufferSegment(blockId, offset, length, uncompressLength, crc, taskAttemptId); + ShuffleSegment segment1 = new ShuffleSegment(0, 1, 2, 3, 4, 5); + ShuffleSegment segment2 = + new ShuffleSegment(blockId, offset, length, uncompressLength, crc, taskAttemptId); assertNotEquals(segment1, segment2); } @Test public void testToString() { - BufferSegment segment = new BufferSegment(0, 1, 2, 3, 4, 5); + ShuffleSegment segment = new ShuffleSegment(0, 1, 2, 3, 4, 5); assertEquals( "BufferSegment{blockId[0], taskAttemptId[5], offset[1], length[2], crc[4], uncompressLength[3]}", segment.toString()); @@ -64,9 +64,9 @@ public void testToString() { @Test public void testGetOffset() { - BufferSegment segment1 = new BufferSegment(0, Integer.MAX_VALUE, 2, 3, 4, 5); + ShuffleSegment segment1 = new ShuffleSegment(0, Integer.MAX_VALUE, 2, 3, 4, 5); assertEquals(Integer.MAX_VALUE, segment1.getOffset()); - BufferSegment segment2 = new BufferSegment(0, (long) Integer.MAX_VALUE + 1, 2, 3, 4, 5); + ShuffleSegment segment2 = new ShuffleSegment(0, (long) Integer.MAX_VALUE + 1, 2, 3, 4, 5); assertThrows(RuntimeException.class, segment2::getOffset); } } diff --git a/common/src/test/java/org/apache/uniffle/common/netty/TransportFrameDecoderTest.java b/common/src/test/java/org/apache/uniffle/common/netty/TransportFrameDecoderTest.java index 1f907ebe9b..2cb54ca5c9 100644 --- a/common/src/test/java/org/apache/uniffle/common/netty/TransportFrameDecoderTest.java +++ b/common/src/test/java/org/apache/uniffle/common/netty/TransportFrameDecoderTest.java @@ -28,7 +28,7 @@ import org.junit.jupiter.api.Test; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleBlockInfo; import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.netty.buffer.NettyManagedBuffer; @@ -168,12 +168,12 @@ private GetLocalShuffleIndexResponse generateGetLocalShuffleIndexResponse() { private GetMemoryShuffleDataResponse generateGetMemoryShuffleDataResponse() { byte[] data4 = new byte[] {1, 2, 3, 4, 5}; - List bufferSegments = + List shuffleSegments = Lists.newArrayList( - new BufferSegment(1, 0, 5, 10, 123, 1), new BufferSegment(1, 0, 5, 10, 345, 1)); + new ShuffleSegment(1, 0, 5, 10, 123, 1), new ShuffleSegment(1, 0, 5, 10, 345, 1)); GetMemoryShuffleDataResponse rpcResponse4 = new GetMemoryShuffleDataResponse( - 1, StatusCode.SUCCESS, "", bufferSegments, Unpooled.wrappedBuffer(data4).retain()); + 1, StatusCode.SUCCESS, "", shuffleSegments, Unpooled.wrappedBuffer(data4).retain()); return rpcResponse4; } 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 a370828c38..f0da66fc23 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 @@ -28,7 +28,7 @@ import org.junit.jupiter.api.Test; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleBlockInfo; import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.netty.buffer.NettyManagedBuffer; @@ -271,12 +271,12 @@ public void testGetMemoryShuffleDataRequest() { @Test public void testGetMemoryShuffleDataResponse() { byte[] data = new byte[] {1, 2, 3, 4, 5}; - List bufferSegments = + List shuffleSegments = Lists.newArrayList( - new BufferSegment(1, 0, 5, 10, 123, 1), new BufferSegment(1, 0, 5, 10, 345, 1)); + new ShuffleSegment(1, 0, 5, 10, 123, 1), new ShuffleSegment(1, 0, 5, 10, 345, 1)); GetMemoryShuffleDataResponse getMemoryShuffleDataResponse = new GetMemoryShuffleDataResponse( - 1, StatusCode.SUCCESS, "", bufferSegments, Unpooled.wrappedBuffer(data).retain()); + 1, StatusCode.SUCCESS, "", shuffleSegments, Unpooled.wrappedBuffer(data).retain()); int encodeLength = getMemoryShuffleDataResponse.encodedLength(); ByteBuf byteBuf = Unpooled.buffer(encodeLength, encodeLength); getMemoryShuffleDataResponse.encode(byteBuf); diff --git a/common/src/test/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitterTest.java b/common/src/test/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitterTest.java index 9e0ff1e5c7..64d77066c0 100644 --- a/common/src/test/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitterTest.java +++ b/common/src/test/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitterTest.java @@ -26,7 +26,7 @@ import org.junit.jupiter.params.provider.ValueSource; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataSegment; import org.apache.uniffle.common.ShuffleIndexResult; @@ -62,22 +62,22 @@ public void testDiscontinuousMapTaskIds() { assertEquals(2, dataSegments1.get(0).getBufferSegments().size()); assertEquals(3, dataSegments1.get(1).getBufferSegments().size()); - BufferSegment bufferSegment = dataSegments1.get(0).getBufferSegments().get(0); - assertEquals(0, bufferSegment.getOffset()); - assertEquals(8, bufferSegment.getLength()); - bufferSegment = dataSegments1.get(0).getBufferSegments().get(1); - assertEquals(8, bufferSegment.getOffset()); - assertEquals(8, bufferSegment.getLength()); - - bufferSegment = dataSegments1.get(1).getBufferSegments().get(0); - assertEquals(0, bufferSegment.getOffset()); - assertEquals(8, bufferSegment.getLength()); - bufferSegment = dataSegments1.get(1).getBufferSegments().get(1); - assertEquals(8, bufferSegment.getOffset()); - assertEquals(8, bufferSegment.getLength()); - bufferSegment = dataSegments1.get(1).getBufferSegments().get(2); - assertEquals(16, bufferSegment.getOffset()); - assertEquals(8, bufferSegment.getLength()); + ShuffleSegment shuffleSegment = dataSegments1.get(0).getBufferSegments().get(0); + assertEquals(0, shuffleSegment.getOffset()); + assertEquals(8, shuffleSegment.getLength()); + shuffleSegment = dataSegments1.get(0).getBufferSegments().get(1); + assertEquals(8, shuffleSegment.getOffset()); + assertEquals(8, shuffleSegment.getLength()); + + shuffleSegment = dataSegments1.get(1).getBufferSegments().get(0); + assertEquals(0, shuffleSegment.getOffset()); + assertEquals(8, shuffleSegment.getLength()); + shuffleSegment = dataSegments1.get(1).getBufferSegments().get(1); + assertEquals(8, shuffleSegment.getOffset()); + assertEquals(8, shuffleSegment.getLength()); + shuffleSegment = dataSegments1.get(1).getBufferSegments().get(2); + assertEquals(16, shuffleSegment.getOffset()); + assertEquals(8, shuffleSegment.getLength()); // case2 taskIds = Roaring64NavigableMap.bitmapOf(1, 2, 4); @@ -174,14 +174,14 @@ private void checkConsistency( assertEquals(segment1.getLength(), segment2.getLength()); assertEquals(segment1.getOffset(), segment2.getOffset()); - List bufferSegments1 = segment1.getBufferSegments(); - List bufferSegments2 = segment2.getBufferSegments(); + List shuffleSegments1 = segment1.getBufferSegments(); + List shuffleSegments2 = segment2.getBufferSegments(); - assertEquals(bufferSegments1.size(), bufferSegments2.size()); + assertEquals(shuffleSegments1.size(), shuffleSegments2.size()); - for (int j = 0; j < bufferSegments1.size(); j++) { - BufferSegment bs1 = bufferSegments1.get(j); - BufferSegment bs2 = bufferSegments2.get(j); + for (int j = 0; j < shuffleSegments1.size(); j++) { + ShuffleSegment bs1 = shuffleSegments1.get(j); + ShuffleSegment bs2 = shuffleSegments2.get(j); assertEquals(bs1.getLength(), bs2.getLength()); assertEquals(bs1.getOffset(), bs2.getOffset()); assertEquals(bs1.getBlockId(), bs2.getBlockId()); @@ -218,28 +218,28 @@ public void testSplitForMergeContinuousSegments() { assertEquals(32, dataSegments.get(0).getOffset()); assertEquals(56, dataSegments.get(0).getLength()); - List bufferSegments = dataSegments.get(0).getBufferSegments(); - assertEquals(0, bufferSegments.get(0).getOffset()); - assertEquals(16, bufferSegments.get(0).getLength()); + List shuffleSegments = dataSegments.get(0).getBufferSegments(); + assertEquals(0, shuffleSegments.get(0).getOffset()); + assertEquals(16, shuffleSegments.get(0).getLength()); - assertEquals(16, bufferSegments.get(1).getOffset()); - assertEquals(10, bufferSegments.get(1).getLength()); + assertEquals(16, shuffleSegments.get(1).getOffset()); + assertEquals(10, shuffleSegments.get(1).getLength()); - assertEquals(26, bufferSegments.get(2).getOffset()); - assertEquals(16, bufferSegments.get(2).getLength()); + assertEquals(26, shuffleSegments.get(2).getOffset()); + assertEquals(16, shuffleSegments.get(2).getLength()); - assertEquals(42, bufferSegments.get(3).getOffset()); - assertEquals(6, bufferSegments.get(3).getLength()); + assertEquals(42, shuffleSegments.get(3).getOffset()); + assertEquals(6, shuffleSegments.get(3).getLength()); - assertEquals(48, bufferSegments.get(4).getOffset()); - assertEquals(8, bufferSegments.get(4).getLength()); + assertEquals(48, shuffleSegments.get(4).getOffset()); + assertEquals(8, shuffleSegments.get(4).getLength()); assertEquals(98, dataSegments.get(1).getOffset()); assertEquals(9, dataSegments.get(1).getLength()); - bufferSegments = dataSegments.get(1).getBufferSegments(); - assertEquals(1, bufferSegments.size()); - assertEquals(0, bufferSegments.get(0).getOffset()); - assertEquals(9, bufferSegments.get(0).getLength()); + shuffleSegments = dataSegments.get(1).getBufferSegments(); + assertEquals(1, shuffleSegments.size()); + assertEquals(0, shuffleSegments.get(0).getOffset()); + assertEquals(9, shuffleSegments.get(0).getLength()); /** * case2: (16, 1) (16, 2) (6, 1) diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerFaultToleranceTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerFaultToleranceTest.java index d08bf80bd7..a3e0096feb 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerFaultToleranceTest.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerFaultToleranceTest.java @@ -40,7 +40,7 @@ import org.apache.uniffle.client.request.RssRegisterShuffleRequest; import org.apache.uniffle.client.request.RssSendCommitRequest; import org.apache.uniffle.client.request.RssSendShuffleDataRequest; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ClientType; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.ShuffleBlockInfo; @@ -168,7 +168,7 @@ private void testReadFaultTolerance(boolean isNettyMode) throws Exception { }); ShuffleDataResult sdr = clientReadHandler.readShuffleData(); TestUtils.validateResult(expectedData, sdr); - for (BufferSegment bs : sdr.getBufferSegments()) { + for (ShuffleSegment bs : sdr.getBufferSegments()) { clientReadHandler.updateConsumedBlockInfo(bs, false); } ClientReadHandlerMetric exceptMetric = mock(ClientReadHandlerMetric.class); @@ -203,7 +203,7 @@ private void testReadFaultTolerance(boolean isNettyMode) throws Exception { expectedData.put(block.getBlockId(), ByteBufUtils.readBytes(block.getData())); }); TestUtils.validateResult(expectedData, sdr); - for (BufferSegment bs : sdr.getBufferSegments()) { + for (ShuffleSegment bs : sdr.getBufferSegments()) { clientReadHandler.updateConsumedBlockInfo(bs, false); } readHandlerMetric = clientReadHandler.getReadHandlerMetric(); @@ -237,7 +237,7 @@ private void testReadFaultTolerance(boolean isNettyMode) throws Exception { ShuffleHandlerFactory.getInstance().createShuffleReadHandler(request); sdr = clientReadHandler.readShuffleData(); TestUtils.validateResult(expectedData, sdr); - for (BufferSegment bs : sdr.getBufferSegments()) { + for (ShuffleSegment bs : sdr.getBufferSegments()) { clientReadHandler.updateConsumedBlockInfo(bs, false); } readHandlerMetric = clientReadHandler.getReadHandlerMetric(); diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithLocalOfLocalOrderTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithLocalOfLocalOrderTest.java index b0ed66d7f9..e6d47f27cc 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithLocalOfLocalOrderTest.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithLocalOfLocalOrderTest.java @@ -47,7 +47,7 @@ import org.apache.uniffle.client.request.RssSendShuffleDataRequest; import org.apache.uniffle.client.response.RssSendShuffleDataResponse; import org.apache.uniffle.client.util.DefaultIdHelper; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.RemoteStorageInfo; import org.apache.uniffle.common.ShuffleBlockInfo; @@ -333,9 +333,9 @@ private void validate( Map expectedData, Set expectedTaskAttemptIds) { byte[] buffer = sdr.getData(); - List bufferSegments = sdr.getBufferSegments(); + List shuffleSegments = sdr.getBufferSegments(); int matched = 0; - for (BufferSegment bs : bufferSegments) { + for (ShuffleSegment bs : shuffleSegments) { if (expectedBlockIds.contains(bs.getBlockId())) { byte[] data = new byte[bs.getLength()]; System.arraycopy(buffer, bs.getOffset(), data, 0, bs.getLength()); diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithLocalTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithLocalTest.java index a77472e6a8..90728735bf 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithLocalTest.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithLocalTest.java @@ -44,7 +44,7 @@ import org.apache.uniffle.client.request.RssSendCommitRequest; import org.apache.uniffle.client.request.RssSendShuffleDataRequest; import org.apache.uniffle.client.response.RssSendShuffleDataResponse; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.ShuffleBlockInfo; import org.apache.uniffle.common.ShuffleDataResult; @@ -191,9 +191,9 @@ protected void validateResult( Map expectedData, long expectedTaskAttemptId) { byte[] buffer = sdr.getData(); - List bufferSegments = sdr.getBufferSegments(); + List shuffleSegments = sdr.getBufferSegments(); int matched = 0; - for (BufferSegment bs : bufferSegments) { + for (ShuffleSegment bs : shuffleSegments) { if (expectedBlockIds.contains(bs.getBlockId())) { byte[] data = new byte[bs.getLength()]; System.arraycopy(buffer, bs.getOffset(), data, 0, bs.getLength()); diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithMemLocalHadoopTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithMemLocalHadoopTest.java index f969397f80..ad73ba8a83 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithMemLocalHadoopTest.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithMemLocalHadoopTest.java @@ -42,7 +42,7 @@ import org.apache.uniffle.client.request.RssRegisterShuffleRequest; import org.apache.uniffle.client.request.RssSendShuffleDataRequest; import org.apache.uniffle.client.response.RssSendShuffleDataResponse; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.ShuffleBlockInfo; import org.apache.uniffle.common.ShuffleDataResult; @@ -355,18 +355,18 @@ protected void waitFlush(String appId, int shuffleId, boolean isNettyMode) protected void validateResult(Map expectedData, ShuffleDataResult sdr) { byte[] buffer = sdr.getData(); - List bufferSegments = sdr.getBufferSegments(); - assertEquals(expectedData.size(), bufferSegments.size()); + List shuffleSegments = sdr.getBufferSegments(); + assertEquals(expectedData.size(), shuffleSegments.size()); for (Map.Entry entry : expectedData.entrySet()) { - BufferSegment bs = findBufferSegment(entry.getKey(), bufferSegments); + ShuffleSegment bs = findBufferSegment(entry.getKey(), shuffleSegments); assertNotNull(bs); byte[] data = new byte[bs.getLength()]; System.arraycopy(buffer, bs.getOffset(), data, 0, bs.getLength()); } } - private BufferSegment findBufferSegment(long blockId, List bufferSegments) { - for (BufferSegment bs : bufferSegments) { + private ShuffleSegment findBufferSegment(long blockId, List shuffleSegments) { + for (ShuffleSegment bs : shuffleSegments) { if (bs.getBlockId() == blockId) { return bs; } diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithMemoryTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithMemoryTest.java index a2de8f7611..fdb2c688aa 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithMemoryTest.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithMemoryTest.java @@ -40,7 +40,7 @@ import org.apache.uniffle.client.request.RssRegisterShuffleRequest; import org.apache.uniffle.client.request.RssSendShuffleDataRequest; import org.apache.uniffle.client.response.RssSendShuffleDataResponse; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.ShuffleBlockInfo; import org.apache.uniffle.common.ShuffleDataResult; @@ -497,18 +497,18 @@ private void memoryAndLocalFileReadWithFilterTest(boolean isNettyMode) throws Ex protected void validateResult(Map expectedData, ShuffleDataResult sdr) { byte[] buffer = sdr.getData(); - List bufferSegments = sdr.getBufferSegments(); - assertEquals(expectedData.size(), bufferSegments.size()); + List shuffleSegments = sdr.getBufferSegments(); + assertEquals(expectedData.size(), shuffleSegments.size()); for (Map.Entry entry : expectedData.entrySet()) { - BufferSegment bs = findBufferSegment(entry.getKey(), bufferSegments); + ShuffleSegment bs = findBufferSegment(entry.getKey(), shuffleSegments); assertNotNull(bs); byte[] data = new byte[bs.getLength()]; System.arraycopy(buffer, bs.getOffset(), data, 0, bs.getLength()); } } - private BufferSegment findBufferSegment(long blockId, List bufferSegments) { - for (BufferSegment bs : bufferSegments) { + private ShuffleSegment findBufferSegment(long blockId, List shuffleSegments) { + for (ShuffleSegment bs : shuffleSegments) { if (bs.getBlockId() == blockId) { return bs; } diff --git a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerGrpcClient.java b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerGrpcClient.java index 988b7a7f05..86e5f0c5b1 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerGrpcClient.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerGrpcClient.java @@ -61,7 +61,7 @@ import org.apache.uniffle.client.response.RssSendShuffleDataResponse; import org.apache.uniffle.client.response.RssUnregisterShuffleByAppIdResponse; import org.apache.uniffle.client.response.RssUnregisterShuffleResponse; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.RemoteStorageInfo; import org.apache.uniffle.common.ShuffleBlockInfo; @@ -1145,11 +1145,11 @@ private List toShufflePartitionRanges( return ret; } - protected List toBufferSegments(List blockSegments) { - List ret = Lists.newArrayList(); + protected List toBufferSegments(List blockSegments) { + List ret = Lists.newArrayList(); for (ShuffleDataBlockSegment sdbs : blockSegments) { ret.add( - new BufferSegment( + new ShuffleSegment( sdbs.getBlockId(), sdbs.getOffset(), sdbs.getLength(), diff --git a/internal-client/src/main/java/org/apache/uniffle/client/response/RssGetInMemoryShuffleDataResponse.java b/internal-client/src/main/java/org/apache/uniffle/client/response/RssGetInMemoryShuffleDataResponse.java index bbf3738cb5..35308a2dc1 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/response/RssGetInMemoryShuffleDataResponse.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/response/RssGetInMemoryShuffleDataResponse.java @@ -22,7 +22,7 @@ import io.netty.buffer.Unpooled; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.netty.buffer.ManagedBuffer; import org.apache.uniffle.common.netty.buffer.NettyManagedBuffer; import org.apache.uniffle.common.rpc.StatusCode; @@ -30,17 +30,17 @@ public class RssGetInMemoryShuffleDataResponse extends ClientResponse { private final ManagedBuffer data; - private final List bufferSegments; + private final List shuffleSegments; public RssGetInMemoryShuffleDataResponse( - StatusCode statusCode, ByteBuffer data, List bufferSegments) { - this(statusCode, new NettyManagedBuffer(Unpooled.wrappedBuffer(data)), bufferSegments); + StatusCode statusCode, ByteBuffer data, List shuffleSegments) { + this(statusCode, new NettyManagedBuffer(Unpooled.wrappedBuffer(data)), shuffleSegments); } public RssGetInMemoryShuffleDataResponse( - StatusCode statusCode, ManagedBuffer data, List bufferSegments) { + StatusCode statusCode, ManagedBuffer data, List shuffleSegments) { super(statusCode); - this.bufferSegments = bufferSegments; + this.shuffleSegments = shuffleSegments; this.data = data; } @@ -48,7 +48,7 @@ public ManagedBuffer getData() { return data; } - public List getBufferSegments() { - return bufferSegments; + public List getBufferSegments() { + return shuffleSegments; } } diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServerGrpcService.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServerGrpcService.java index aea43d24e7..bcb9739041 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleServerGrpcService.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServerGrpcService.java @@ -37,7 +37,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.RemoteStorageInfo; import org.apache.uniffle.common.ShuffleDataDistributionType; @@ -967,10 +967,10 @@ public void getMemoryShuffleData( .getInMemoryShuffleData( appId, shuffleId, partitionId, blockId, readBufferSize, expectedTaskIds); byte[] data = new byte[] {}; - List bufferSegments = Lists.newArrayList(); + List shuffleSegments = Lists.newArrayList(); if (shuffleDataResult != null) { data = shuffleDataResult.getData(); - bufferSegments = shuffleDataResult.getBufferSegments(); + shuffleSegments = shuffleDataResult.getBufferSegments(); ShuffleServerMetrics.counterTotalReadDataSize.inc(data.length); ShuffleServerMetrics.counterTotalReadMemoryDataSize.inc(data.length); ShuffleServerMetrics.gaugeReadMemoryDataThreadNum.inc(); @@ -991,7 +991,7 @@ public void getMemoryShuffleData( .setStatus(status.toProto()) .setRetMsg(msg) .setData(UnsafeByteOperations.unsafeWrap(data)) - .addAllShuffleDataBlockSegments(toShuffleDataBlockSegments(bufferSegments)) + .addAllShuffleDataBlockSegments(toShuffleDataBlockSegments(shuffleSegments)) .build(); } catch (Exception e) { status = StatusCode.INTERNAL_ERROR; @@ -1091,10 +1091,10 @@ private List toPartitionRanges( } private List toShuffleDataBlockSegments( - List bufferSegments) { + List shuffleSegments) { List shuffleDataBlockSegments = Lists.newArrayList(); - if (bufferSegments != null) { - for (BufferSegment bs : bufferSegments) { + if (shuffleSegments != null) { + for (ShuffleSegment bs : shuffleSegments) { shuffleDataBlockSegments.add( ShuffleDataBlockSegment.newBuilder() .setBlockId(bs.getBlockId()) diff --git a/server/src/main/java/org/apache/uniffle/server/buffer/AbstractShuffleBuffer.java b/server/src/main/java/org/apache/uniffle/server/buffer/AbstractShuffleBuffer.java index ddbeb21cf2..7f1ddc309b 100644 --- a/server/src/main/java/org/apache/uniffle/server/buffer/AbstractShuffleBuffer.java +++ b/server/src/main/java/org/apache/uniffle/server/buffer/AbstractShuffleBuffer.java @@ -27,7 +27,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataDistributionType; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShufflePartitionedBlock; @@ -79,11 +79,11 @@ public synchronized ShuffleDataResult getShuffleData(long lastBlockId, int readB public synchronized ShuffleDataResult getShuffleData( long lastBlockId, int readBufferSize, Roaring64NavigableMap expectedTaskIds) { try { - List bufferSegments = Lists.newArrayList(); + List shuffleSegments = Lists.newArrayList(); List readBlocks = Lists.newArrayList(); updateBufferSegmentsAndResultBlocks( - lastBlockId, readBufferSize, bufferSegments, readBlocks, expectedTaskIds); - if (!bufferSegments.isEmpty()) { + lastBlockId, readBufferSize, shuffleSegments, readBlocks, expectedTaskIds); + if (!shuffleSegments.isEmpty()) { CompositeByteBuf byteBuf = new CompositeByteBuf( NettyUtils.getSharedUnpooledByteBufAllocator(true), @@ -91,7 +91,7 @@ public synchronized ShuffleDataResult getShuffleData( Constants.COMPOSITE_BYTE_BUF_MAX_COMPONENTS); // copy result data updateShuffleData(readBlocks, byteBuf); - return new ShuffleDataResult(byteBuf, bufferSegments); + return new ShuffleDataResult(byteBuf, shuffleSegments); } } catch (Exception e) { LOG.error("Exception happened when getShuffleData in buffer", e); @@ -105,13 +105,13 @@ public synchronized ShuffleDataResult getShuffleData( protected abstract void updateBufferSegmentsAndResultBlocks( long lastBlockId, long readBufferSize, - List bufferSegments, + List shuffleSegments, List resultBlocks, Roaring64NavigableMap expectedTaskIds); - protected int calculateDataLength(List bufferSegments) { - BufferSegment bufferSegment = bufferSegments.get(bufferSegments.size() - 1); - return bufferSegment.getOffset() + bufferSegment.getLength(); + protected int calculateDataLength(List shuffleSegments) { + ShuffleSegment shuffleSegment = shuffleSegments.get(shuffleSegments.size() - 1); + return shuffleSegment.getOffset() + shuffleSegment.getLength(); } private void updateShuffleData(List readBlocks, CompositeByteBuf data) { @@ -151,7 +151,7 @@ protected void updateSegmentsWithoutBlockId( int offset, Collection cachedBlocks, long readBufferSize, - List bufferSegments, + List shuffleSegments, List readBlocks, Roaring64NavigableMap expectedTaskIds) { int currentOffset = offset; @@ -161,8 +161,8 @@ protected void updateSegmentsWithoutBlockId( continue; } // add bufferSegment with block - bufferSegments.add( - new BufferSegment( + shuffleSegments.add( + new ShuffleSegment( block.getBlockId(), currentOffset, block.getLength(), diff --git a/server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferWithLinkedList.java b/server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferWithLinkedList.java index 3f5ff900c1..409a8f9227 100644 --- a/server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferWithLinkedList.java +++ b/server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferWithLinkedList.java @@ -26,7 +26,7 @@ import com.google.common.collect.Lists; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataDistributionType; import org.apache.uniffle.common.ShufflePartitionedBlock; import org.apache.uniffle.common.ShufflePartitionedData; @@ -128,7 +128,7 @@ public Map> getInFlushBlockMap() { protected void updateBufferSegmentsAndResultBlocks( long lastBlockId, long readBufferSize, - List bufferSegments, + List shuffleSegments, List resultBlocks, Roaring64NavigableMap expectedTaskIds) { long nextBlockId = lastBlockId; @@ -148,7 +148,7 @@ protected void updateBufferSegmentsAndResultBlocks( offset, inFlushBlockMap.get(eventId), readBufferSize, - bufferSegments, + shuffleSegments, resultBlocks, expectedTaskIds); hasLastBlockId = true; @@ -159,7 +159,7 @@ protected void updateBufferSegmentsAndResultBlocks( inFlushBlockMap.get(eventId), readBufferSize, nextBlockId, - bufferSegments, + shuffleSegments, resultBlocks, expectedTaskIds); // if last blockId is found, read from begin with next cached blocks @@ -168,8 +168,8 @@ protected void updateBufferSegmentsAndResultBlocks( nextBlockId = Constants.INVALID_BLOCK_ID; } } - if (!bufferSegments.isEmpty()) { - offset = calculateDataLength(bufferSegments); + if (!shuffleSegments.isEmpty()) { + offset = calculateDataLength(shuffleSegments); } if (offset >= readBufferSize) { break; @@ -180,7 +180,7 @@ protected void updateBufferSegmentsAndResultBlocks( if (blocks.size() > 0 && offset < readBufferSize) { if (nextBlockId == Constants.INVALID_BLOCK_ID) { updateSegmentsWithoutBlockId( - offset, blocks, readBufferSize, bufferSegments, resultBlocks, expectedTaskIds); + offset, blocks, readBufferSize, shuffleSegments, resultBlocks, expectedTaskIds); hasLastBlockId = true; } else { hasLastBlockId = @@ -189,7 +189,7 @@ protected void updateBufferSegmentsAndResultBlocks( blocks, readBufferSize, nextBlockId, - bufferSegments, + shuffleSegments, resultBlocks, expectedTaskIds); } @@ -201,7 +201,7 @@ protected void updateBufferSegmentsAndResultBlocks( updateBufferSegmentsAndResultBlocks( Constants.INVALID_BLOCK_ID, readBufferSize, - bufferSegments, + shuffleSegments, resultBlocks, expectedTaskIds); } @@ -212,7 +212,7 @@ private boolean updateSegmentsWithBlockId( List cachedBlocks, long readBufferSize, long lastBlockId, - List bufferSegments, + List shuffleSegments, List readBlocks, Roaring64NavigableMap expectedTaskIds) { int currentOffset = offset; @@ -230,8 +230,8 @@ private boolean updateSegmentsWithBlockId( continue; } // add bufferSegment with block - bufferSegments.add( - new BufferSegment( + shuffleSegments.add( + new ShuffleSegment( block.getBlockId(), currentOffset, block.getLength(), diff --git a/server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferWithSkipList.java b/server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferWithSkipList.java index e7630b4347..fc9c06947e 100644 --- a/server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferWithSkipList.java +++ b/server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferWithSkipList.java @@ -30,7 +30,7 @@ import com.google.common.collect.Lists; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataDistributionType; import org.apache.uniffle.common.ShufflePartitionedBlock; import org.apache.uniffle.common.ShufflePartitionedData; @@ -130,7 +130,7 @@ public Map> getInFlushBlockMap() { protected void updateBufferSegmentsAndResultBlocks( long lastBlockId, long readBufferSize, - List bufferSegments, + List shuffleSegments, List resultBlocks, Roaring64NavigableMap expectedTaskIds) { long nextBlockId = lastBlockId; @@ -150,7 +150,7 @@ protected void updateBufferSegmentsAndResultBlocks( inFlushBlockMap.get(eventId), readBufferSize, nextBlockId, - bufferSegments, + shuffleSegments, resultBlocks, expectedTaskIds); // if last blockId is found, read from begin with next cached blocks @@ -158,8 +158,8 @@ protected void updateBufferSegmentsAndResultBlocks( // reset blockId to read from begin in next cached blocks nextBlockId = Constants.INVALID_BLOCK_ID; } - if (!bufferSegments.isEmpty()) { - offset = calculateDataLength(bufferSegments); + if (!shuffleSegments.isEmpty()) { + offset = calculateDataLength(shuffleSegments); } if (offset >= readBufferSize) { break; @@ -174,7 +174,7 @@ protected void updateBufferSegmentsAndResultBlocks( blocksMap, readBufferSize, nextBlockId, - bufferSegments, + shuffleSegments, resultBlocks, expectedTaskIds); } @@ -185,7 +185,7 @@ protected void updateBufferSegmentsAndResultBlocks( updateBufferSegmentsAndResultBlocks( Constants.INVALID_BLOCK_ID, readBufferSize, - bufferSegments, + shuffleSegments, resultBlocks, expectedTaskIds); } @@ -196,7 +196,7 @@ private boolean updateSegments( ConcurrentSkipListMap cachedBlocks, long readBufferSize, long lastBlockId, - List bufferSegments, + List shuffleSegments, List readBlocks, Roaring64NavigableMap expectedTaskIds) { int currentOffset = offset; @@ -217,8 +217,8 @@ private boolean updateSegments( continue; } // add bufferSegment with block - bufferSegments.add( - new BufferSegment( + shuffleSegments.add( + new ShuffleSegment( block.getBlockId(), currentOffset, block.getLength(), 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 27a3f1dc48..ada8444a45 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 @@ -31,7 +31,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleBlockInfo; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShuffleIndexResult; @@ -371,17 +371,17 @@ public void handleGetMemoryShuffleDataRequest( readBufferSize, req.getExpectedTaskIdsBitmap()); ManagedBuffer data = NettyManagedBuffer.EMPTY_BUFFER; - List bufferSegments = Lists.newArrayList(); + List shuffleSegments = Lists.newArrayList(); if (shuffleDataResult != null) { data = shuffleDataResult.getManagedBuffer(); - bufferSegments = shuffleDataResult.getBufferSegments(); + shuffleSegments = shuffleDataResult.getBufferSegments(); ShuffleServerMetrics.counterTotalReadDataSize.inc(data.size()); ShuffleServerMetrics.counterTotalReadMemoryDataSize.inc(data.size()); ShuffleServerMetrics.gaugeReadMemoryDataThreadNum.inc(); ShuffleServerMetrics.gaugeReadMemoryDataBufferSize.inc(readBufferSize); } response = - new GetMemoryShuffleDataResponse(req.getRequestId(), status, msg, bufferSegments, data); + new GetMemoryShuffleDataResponse(req.getRequestId(), status, msg, shuffleSegments, data); ReleaseMemoryAndRecordReadTimeListener listener = new ReleaseMemoryAndRecordReadTimeListener( start, readBufferSize, data.size(), requestInfo, req, response, client); diff --git a/server/src/test/java/org/apache/uniffle/server/ShuffleFlushManagerTest.java b/server/src/test/java/org/apache/uniffle/server/ShuffleFlushManagerTest.java index 2b21445e32..573aeedc31 100644 --- a/server/src/test/java/org/apache/uniffle/server/ShuffleFlushManagerTest.java +++ b/server/src/test/java/org/apache/uniffle/server/ShuffleFlushManagerTest.java @@ -47,7 +47,7 @@ import org.junit.jupiter.api.io.TempDir; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.RemoteStorageInfo; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShufflePartitionedBlock; @@ -693,16 +693,16 @@ private void validate( ShuffleDataResult sdr = null; int matchNum = 0; sdr = handler.readShuffleData(); - List bufferSegments = sdr.getBufferSegments(); + List shuffleSegments = sdr.getBufferSegments(); for (ShufflePartitionedBlock block : blocks) { - for (BufferSegment bs : bufferSegments) { + for (ShuffleSegment bs : shuffleSegments) { if (bs.getBlockId() == block.getBlockId()) { matchNum++; break; } } } - for (BufferSegment bs : bufferSegments) { + for (ShuffleSegment bs : shuffleSegments) { remainIds.remove(bs.getBlockId()); } assertEquals(blocks.size(), matchNum); diff --git a/server/src/test/java/org/apache/uniffle/server/ShuffleTaskManagerTest.java b/server/src/test/java/org/apache/uniffle/server/ShuffleTaskManagerTest.java index 75c49cd4dc..7d1d14c4ea 100644 --- a/server/src/test/java/org/apache/uniffle/server/ShuffleTaskManagerTest.java +++ b/server/src/test/java/org/apache/uniffle/server/ShuffleTaskManagerTest.java @@ -43,7 +43,7 @@ import org.junit.jupiter.api.io.TempDir; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.RemoteStorageInfo; import org.apache.uniffle.common.ShuffleDataResult; @@ -1123,10 +1123,10 @@ private void validate( new Configuration()); ShuffleDataResult sdr = handler.readShuffleData(); - List bufferSegments = sdr.getBufferSegments(); + List shuffleSegments = sdr.getBufferSegments(); int matchNum = 0; for (ShufflePartitionedBlock block : blocks) { - for (BufferSegment bs : bufferSegments) { + for (ShuffleSegment bs : shuffleSegments) { if (bs.getBlockId() == block.getBlockId()) { assertEquals(block.getLength(), bs.getLength()); assertEquals(block.getCrc(), bs.getCrc()); diff --git a/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferWithLinkedListTest.java b/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferWithLinkedListTest.java index b262b7c32f..fae66ee75e 100644 --- a/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferWithLinkedListTest.java +++ b/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferWithLinkedListTest.java @@ -24,7 +24,7 @@ import org.junit.jupiter.api.Test; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataDistributionType; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShufflePartitionedBlock; @@ -98,7 +98,7 @@ public void getShuffleDataWithExpectedTaskIdsFilterTest() { ShuffleDataResult result = shuffleBuffer.getShuffleData(Constants.INVALID_BLOCK_ID, 1000, expectedTasks); assertEquals(3, result.getBufferSegments().size()); - for (BufferSegment segment : result.getBufferSegments()) { + for (ShuffleSegment segment : result.getBufferSegments()) { assertTrue(expectedTasks.contains(segment.getTaskAttemptId())); } assertEquals(0, result.getBufferSegments().get(0).getOffset()); @@ -140,7 +140,7 @@ public void getShuffleDataWithExpectedTaskIdsFilterTest() { shuffleBuffer.toFlushEvent("appId", 0, 0, 1, null, ShuffleDataDistributionType.LOCAL_ORDER); result = shuffleBuffer.getShuffleData(Constants.INVALID_BLOCK_ID, 1000, expectedTasks); assertEquals(3, result.getBufferSegments().size()); - for (BufferSegment segment : result.getBufferSegments()) { + for (ShuffleSegment segment : result.getBufferSegments()) { assertTrue(expectedTasks.contains(segment.getTaskAttemptId())); } assertEquals(0, result.getBufferSegments().get(0).getOffset()); @@ -578,15 +578,15 @@ private byte[] getExpectedData(ShufflePartitionedData... spds) { private void compareBufferSegment( List blocks, - List bufferSegments, + List shuffleSegments, int startBlockIndex, int expectedBlockNum) { int segmentIndex = 0; int offset = 0; - assertEquals(expectedBlockNum, bufferSegments.size()); + assertEquals(expectedBlockNum, shuffleSegments.size()); for (int i = startBlockIndex; i < startBlockIndex + expectedBlockNum; i++) { ShufflePartitionedBlock spb = blocks.get(i); - BufferSegment segment = bufferSegments.get(segmentIndex); + ShuffleSegment segment = shuffleSegments.get(segmentIndex); assertEquals(spb.getBlockId(), segment.getBlockId()); assertEquals(spb.getLength(), segment.getLength()); assertEquals(spb.getCrc(), segment.getCrc()); diff --git a/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferWithSkipListTest.java b/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferWithSkipListTest.java index 5bb5e2aa1f..25637717b4 100644 --- a/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferWithSkipListTest.java +++ b/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferWithSkipListTest.java @@ -22,7 +22,7 @@ import org.junit.jupiter.api.Test; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataDistributionType; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShufflePartitionedBlock; @@ -93,7 +93,7 @@ public void getShuffleDataWithExpectedTaskIdsFilterTest() { ShuffleDataResult result = shuffleBuffer.getShuffleData(Constants.INVALID_BLOCK_ID, 1000, expectedTasks); assertEquals(3, result.getBufferSegments().size()); - for (BufferSegment segment : result.getBufferSegments()) { + for (ShuffleSegment segment : result.getBufferSegments()) { assertTrue(expectedTasks.contains(segment.getTaskAttemptId())); } assertEquals(0, result.getBufferSegments().get(0).getOffset()); @@ -139,7 +139,7 @@ public void getShuffleDataWithExpectedTaskIdsFilterTest() { shuffleBuffer.toFlushEvent("appId", 0, 0, 1, null, ShuffleDataDistributionType.LOCAL_ORDER); result = shuffleBuffer.getShuffleData(Constants.INVALID_BLOCK_ID, 1000, expectedTasks); assertEquals(3, result.getBufferSegments().size()); - for (BufferSegment segment : result.getBufferSegments()) { + for (ShuffleSegment segment : result.getBufferSegments()) { assertTrue(expectedTasks.contains(segment.getTaskAttemptId())); } assertEquals(0, result.getBufferSegments().get(0).getOffset()); diff --git a/storage/src/main/java/org/apache/uniffle/storage/api/FileWriter.java b/storage/src/main/java/org/apache/uniffle/storage/api/FileWriter.java index d61b756d4a..d5961b48a6 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/api/FileWriter.java +++ b/storage/src/main/java/org/apache/uniffle/storage/api/FileWriter.java @@ -19,11 +19,11 @@ import java.io.IOException; -import org.apache.uniffle.storage.common.FileBasedShuffleSegment; +import org.apache.uniffle.common.ShuffleSegment; public interface FileWriter { void writeData(byte[] data) throws IOException; - void writeIndex(FileBasedShuffleSegment segment) throws IOException; + void writeIndex(ShuffleSegment segment) throws IOException; } diff --git a/storage/src/main/java/org/apache/uniffle/storage/common/FileBasedShuffleSegment.java b/storage/src/main/java/org/apache/uniffle/storage/common/FileBasedShuffleSegment.java deleted file mode 100644 index 625c128558..0000000000 --- a/storage/src/main/java/org/apache/uniffle/storage/common/FileBasedShuffleSegment.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * 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.storage.common; - -import java.util.Objects; - -public class FileBasedShuffleSegment extends ShuffleSegment - implements Comparable { - - public static final int SEGMENT_SIZE = 4 * Long.BYTES + 2 * Integer.BYTES; - private long offset; - private int length; - private int uncompressLength; - private long crc; - private long blockId; - private long taskAttemptId; - - public FileBasedShuffleSegment( - long blockId, long offset, int length, int uncompressLength, long crc, long taskAttemptId) { - this.offset = offset; - this.length = length; - this.uncompressLength = uncompressLength; - this.crc = crc; - this.blockId = blockId; - this.taskAttemptId = taskAttemptId; - } - - public long getOffset() { - return offset; - } - - public void setOffset(long offset) { - this.offset = offset; - } - - public int getLength() { - return length; - } - - public void setLength(int length) { - this.length = length; - } - - public long getCrc() { - return crc; - } - - public long getBlockId() { - return blockId; - } - - public void setBlockId(long blockId) { - this.blockId = blockId; - } - - public int getUncompressLength() { - return uncompressLength; - } - - public long getTaskAttemptId() { - return taskAttemptId; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - FileBasedShuffleSegment that = (FileBasedShuffleSegment) o; - return offset == that.offset - && length == that.length - && crc == that.crc - && blockId == that.blockId - && uncompressLength == that.uncompressLength - && taskAttemptId == that.taskAttemptId; - } - - @Override - public int compareTo(FileBasedShuffleSegment fss) { - if (this.offset > fss.getOffset()) { - return 1; - } else if (this.offset < fss.getOffset()) { - return -1; - } - return 0; - } - - @Override - public int hashCode() { - return Objects.hash(offset, length, uncompressLength, crc, blockId); - } - - @Override - public String toString() { - return "FileBasedShuffleSegment{" - + "offset[" - + offset - + "], length[" - + length - + "], uncompressLength[" - + uncompressLength - + "], crc[" - + crc - + "], blockid[" - + blockId - + "], taskAttemptId[" - + taskAttemptId - + "]}"; - } -} diff --git a/storage/src/main/java/org/apache/uniffle/storage/common/ShuffleSegment.java b/storage/src/main/java/org/apache/uniffle/storage/common/ShuffleSegment.java deleted file mode 100644 index 4ec2e60147..0000000000 --- a/storage/src/main/java/org/apache/uniffle/storage/common/ShuffleSegment.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * 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.storage.common; - -public abstract class ShuffleSegment implements java.io.Serializable {} diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/api/ClientReadHandler.java b/storage/src/main/java/org/apache/uniffle/storage/handler/api/ClientReadHandler.java index e206a29e1f..1f7bf0bdbf 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/api/ClientReadHandler.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/api/ClientReadHandler.java @@ -17,7 +17,7 @@ package org.apache.uniffle.storage.handler.api; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; public interface ClientReadHandler { @@ -30,7 +30,7 @@ public interface ClientReadHandler { // but does not know the actually consumed blocks, // so the consumer should let the handler update statistics. // Each type of handler can design their rules. - void updateConsumedBlockInfo(BufferSegment bs, boolean isSkippedMetrics); + void updateConsumedBlockInfo(ShuffleSegment bs, boolean isSkippedMetrics); // Display the statistics of consumed blocks void logConsumedBlockInfo(); diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/AbstractClientReadHandler.java b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/AbstractClientReadHandler.java index b3fd6cddf6..7e238a3dbd 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/AbstractClientReadHandler.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/AbstractClientReadHandler.java @@ -20,7 +20,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.storage.handler.ClientReadHandlerMetric; import org.apache.uniffle.storage.handler.api.ClientReadHandler; @@ -42,7 +42,7 @@ public ShuffleDataResult readShuffleData() { public void close() {} @Override - public void updateConsumedBlockInfo(BufferSegment bs, boolean isSkippedMetrics) { + public void updateConsumedBlockInfo(ShuffleSegment bs, boolean isSkippedMetrics) { if (bs == null) { return; } @@ -70,7 +70,7 @@ public void logConsumedBlockInfo() { } protected void updateBlockMetric( - ClientReadHandlerMetric metric, BufferSegment bs, boolean isSkippedMetrics) { + ClientReadHandlerMetric metric, ShuffleSegment bs, boolean isSkippedMetrics) { if (isSkippedMetrics) { metric.incSkippedReadBlockNum(); metric.incSkippedReadLength(bs.getLength()); diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/ComposedClientReadHandler.java b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/ComposedClientReadHandler.java index e619bbd1dd..4a1b6d8407 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/ComposedClientReadHandler.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/ComposedClientReadHandler.java @@ -29,7 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.exception.RssException; @@ -143,7 +143,7 @@ public void close() { } @Override - public void updateConsumedBlockInfo(BufferSegment bs, boolean isSkippedMetrics) { + public void updateConsumedBlockInfo(ShuffleSegment bs, boolean isSkippedMetrics) { if (bs == null) { return; } diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/DataFileSegment.java b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/DataFileSegment.java index bba2ae6c49..57d80c01c4 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/DataFileSegment.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/DataFileSegment.java @@ -22,24 +22,24 @@ import com.google.common.collect.Sets; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; public class DataFileSegment extends FileSegment { - private List bufferSegments; + private List shuffleSegments; - public DataFileSegment(String path, long offset, int length, List bufferSegments) { + public DataFileSegment(String path, long offset, int length, List shuffleSegments) { super(path, offset, length); - this.bufferSegments = bufferSegments; + this.shuffleSegments = shuffleSegments; } - public List getBufferSegments() { - return bufferSegments; + public List getBufferSegments() { + return shuffleSegments; } public Set getBlockIds() { Set blockIds = Sets.newHashSet(); - for (BufferSegment bs : bufferSegments) { + for (ShuffleSegment bs : shuffleSegments) { blockIds.add(bs.getBlockId()); } return blockIds; diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/HadoopFileWriter.java b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/HadoopFileWriter.java index e6074f09d3..43b3df7753 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/HadoopFileWriter.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/HadoopFileWriter.java @@ -27,11 +27,12 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IOUtils; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.storage.api.FileWriter; -import org.apache.uniffle.storage.common.FileBasedShuffleSegment; public class HadoopFileWriter implements FileWriter, Closeable { @@ -94,7 +95,7 @@ public void writeData(ByteBuffer byteBuffer) throws IOException { nextOffset = fsDataOutputStream.getPos(); } - public void writeIndex(FileBasedShuffleSegment segment) throws IOException { + public void writeIndex(ShuffleSegment segment) throws IOException { fsDataOutputStream.writeLong(segment.getOffset()); fsDataOutputStream.writeInt(segment.getLength()); fsDataOutputStream.writeInt(segment.getUncompressLength()); diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/HadoopShuffleReadHandler.java b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/HadoopShuffleReadHandler.java index c7af921b4f..1c353ee33a 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/HadoopShuffleReadHandler.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/HadoopShuffleReadHandler.java @@ -31,7 +31,7 @@ import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShuffleDataSegment; import org.apache.uniffle.common.ShuffleIndexResult; -import org.apache.uniffle.storage.common.FileBasedShuffleSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.storage.util.ShuffleStorageUtils; /** @@ -112,8 +112,8 @@ protected ShuffleIndexResult readShuffleIndex() { indexData = ByteBuffer.wrap(indexReader.read()); } int indexDataLength = indexData.limit() - indexData.position(); - int segmentNumber = indexDataLength / FileBasedShuffleSegment.SEGMENT_SIZE; - int expectedLen = segmentNumber * FileBasedShuffleSegment.SEGMENT_SIZE; + int segmentNumber = indexDataLength / ShuffleSegment.SEGMENT_SIZE; + int expectedLen = segmentNumber * ShuffleSegment.SEGMENT_SIZE; if (indexDataLength != expectedLen) { LOG.warn( "Maybe the index file: {} is being written due to the shuffle-buffer flushing.", diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/HadoopShuffleWriteHandler.java b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/HadoopShuffleWriteHandler.java index eb81c40898..326f70d505 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/HadoopShuffleWriteHandler.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/HadoopShuffleWriteHandler.java @@ -30,10 +30,10 @@ import org.slf4j.LoggerFactory; import org.apache.uniffle.common.ShufflePartitionedBlock; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.exception.RssException; import org.apache.uniffle.common.filesystem.HadoopFilesystemProvider; import org.apache.uniffle.common.util.ByteBufUtils; -import org.apache.uniffle.storage.common.FileBasedShuffleSegment; import org.apache.uniffle.storage.handler.api.ShuffleWriteHandler; import org.apache.uniffle.storage.util.ShuffleStorageUtils; @@ -128,8 +128,8 @@ public void write(List shuffleBlocks) throws Exception long startOffset = dataWriter.nextOffset(); dataWriter.writeData(ByteBufUtils.readBytes(block.getData())); - FileBasedShuffleSegment segment = - new FileBasedShuffleSegment( + ShuffleSegment segment = + new ShuffleSegment( blockId, startOffset, block.getLength(), diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/LocalFileServerReadHandler.java b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/LocalFileServerReadHandler.java index f688a18bcb..b8adc77ade 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/LocalFileServerReadHandler.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/LocalFileServerReadHandler.java @@ -25,11 +25,11 @@ import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShuffleIndexResult; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.exception.FileNotFoundException; import org.apache.uniffle.common.exception.RssException; import org.apache.uniffle.common.netty.buffer.FileSegmentManagedBuffer; import org.apache.uniffle.common.util.Constants; -import org.apache.uniffle.storage.common.FileBasedShuffleSegment; import org.apache.uniffle.storage.handler.api.ServerReadHandler; import org.apache.uniffle.storage.util.ShuffleStorageUtils; @@ -140,8 +140,8 @@ public ShuffleDataResult getShuffleData(long offset, int length) { public ShuffleIndexResult getShuffleIndex() { File indexFile = new File(indexFileName); long indexFileSize = indexFile.length(); - int indexNum = (int) (indexFileSize / FileBasedShuffleSegment.SEGMENT_SIZE); - int len = indexNum * FileBasedShuffleSegment.SEGMENT_SIZE; + int indexNum = (int) (indexFileSize / ShuffleSegment.SEGMENT_SIZE); + int len = indexNum * ShuffleSegment.SEGMENT_SIZE; if (indexFileSize != len) { LOG.warn( "Maybe the index file: {} is being written due to the shuffle-buffer flushing.", diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/LocalFileWriteHandler.java b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/LocalFileWriteHandler.java index 4b06e5aa91..c7693af7ae 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/LocalFileWriteHandler.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/LocalFileWriteHandler.java @@ -27,9 +27,9 @@ import org.slf4j.LoggerFactory; import org.apache.uniffle.common.ShufflePartitionedBlock; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.exception.RssException; import org.apache.uniffle.common.util.ByteBufUtils; -import org.apache.uniffle.storage.common.FileBasedShuffleSegment; import org.apache.uniffle.storage.handler.api.ShuffleWriteHandler; import org.apache.uniffle.storage.util.ShuffleStorageUtils; @@ -105,8 +105,8 @@ public synchronized void write(List shuffleBlocks) thro long startOffset = dataWriter.nextOffset(); dataWriter.writeData(ByteBufUtils.readBytes(block.getData())); - FileBasedShuffleSegment segment = - new FileBasedShuffleSegment( + ShuffleSegment segment = + new ShuffleSegment( blockId, startOffset, block.getLength(), diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/LocalFileWriter.java b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/LocalFileWriter.java index 01c188f3ff..ee56e82469 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/LocalFileWriter.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/LocalFileWriter.java @@ -24,8 +24,8 @@ import java.io.FileOutputStream; import java.io.IOException; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.storage.api.FileWriter; -import org.apache.uniffle.storage.common.FileBasedShuffleSegment; public class LocalFileWriter implements FileWriter, Closeable { @@ -47,7 +47,7 @@ public void writeData(byte[] data) throws IOException { } } - public void writeIndex(FileBasedShuffleSegment segment) throws IOException { + public void writeIndex(ShuffleSegment segment) throws IOException { dataOutputStream.writeLong(segment.getOffset()); dataOutputStream.writeInt(segment.getLength()); dataOutputStream.writeInt(segment.getUncompressLength()); diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/MemoryClientReadHandler.java b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/MemoryClientReadHandler.java index f1fbe2361c..97da5ac4a4 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/MemoryClientReadHandler.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/MemoryClientReadHandler.java @@ -27,7 +27,7 @@ import org.apache.uniffle.client.api.ShuffleServerClient; import org.apache.uniffle.client.request.RssGetInMemoryShuffleDataRequest; import org.apache.uniffle.client.response.RssGetInMemoryShuffleDataResponse; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.exception.RssFetchFailedException; import org.apache.uniffle.common.util.Constants; @@ -100,8 +100,8 @@ public ShuffleDataResult readShuffleData() { // update lastBlockId for next rpc call if (!result.isEmpty()) { - List bufferSegments = result.getBufferSegments(); - lastBlockId = bufferSegments.get(bufferSegments.size() - 1).getBlockId(); + List shuffleSegments = result.getBufferSegments(); + lastBlockId = shuffleSegments.get(shuffleSegments.size() - 1).getBlockId(); } return result; diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/MultiReplicaClientReadHandler.java b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/MultiReplicaClientReadHandler.java index c3e2200f95..a5eef07628 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/MultiReplicaClientReadHandler.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/MultiReplicaClientReadHandler.java @@ -23,7 +23,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.exception.RssException; @@ -87,7 +87,7 @@ public ShuffleDataResult readShuffleData() { } @Override - public void updateConsumedBlockInfo(BufferSegment bs, boolean isSkippedMetrics) { + public void updateConsumedBlockInfo(ShuffleSegment bs, boolean isSkippedMetrics) { super.updateConsumedBlockInfo(bs, isSkippedMetrics); handlers .get(Math.min(readHandlerIndex, handlers.size() - 1)) diff --git a/storage/src/main/java/org/apache/uniffle/storage/util/ShuffleStorageUtils.java b/storage/src/main/java/org/apache/uniffle/storage/util/ShuffleStorageUtils.java index daaf27e387..960394b302 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/util/ShuffleStorageUtils.java +++ b/storage/src/main/java/org/apache/uniffle/storage/util/ShuffleStorageUtils.java @@ -32,10 +32,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.exception.RssException; import org.apache.uniffle.common.util.Constants; -import org.apache.uniffle.storage.common.FileBasedShuffleSegment; import org.apache.uniffle.storage.handler.impl.DataFileSegment; import org.apache.uniffle.storage.handler.impl.HadoopFileWriter; @@ -56,13 +55,13 @@ public static String generateIndexFileName(String fileNamePrefix) { } public static List mergeSegments( - String path, List segments, int readBufferSize) { + String path, List segments, int readBufferSize) { List dataFileSegments = Lists.newArrayList(); if (segments != null && !segments.isEmpty()) { if (segments.size() == 1) { - List bufferSegments = Lists.newArrayList(); - bufferSegments.add( - new BufferSegment( + List shuffleSegments = Lists.newArrayList(); + shuffleSegments.add( + new ShuffleSegment( segments.get(0).getBlockId(), 0, segments.get(0).getLength(), @@ -71,30 +70,30 @@ public static List mergeSegments( segments.get(0).getTaskAttemptId())); dataFileSegments.add( new DataFileSegment( - path, segments.get(0).getOffset(), segments.get(0).getLength(), bufferSegments)); + path, segments.get(0).getOffset(), segments.get(0).getLength(), shuffleSegments)); } else { Collections.sort(segments); long start = -1; long latestPosition = -1; long skipThreshold = readBufferSize / 2; long lastPosition = Long.MAX_VALUE; - List bufferSegments = Lists.newArrayList(); - for (FileBasedShuffleSegment segment : segments) { + List shuffleSegments = Lists.newArrayList(); + for (ShuffleSegment segment : segments) { // check if there has expected skip range, eg, [20, 100], [1000, 1001] and the skip range // is [101, 999] if (start > -1 && segment.getOffset() - lastPosition > skipThreshold) { dataFileSegments.add( - new DataFileSegment(path, start, (int) (lastPosition - start), bufferSegments)); + new DataFileSegment(path, start, (int) (lastPosition - start), shuffleSegments)); start = -1; } - // previous FileBasedShuffleSegment are merged, start new merge process + // previous ShuffleSegment are merged, start new merge process if (start == -1) { - bufferSegments = Lists.newArrayList(); + shuffleSegments = Lists.newArrayList(); start = segment.getOffset(); } latestPosition = segment.getOffset() + segment.getLength(); - bufferSegments.add( - new BufferSegment( + shuffleSegments.add( + new ShuffleSegment( segment.getBlockId(), segment.getOffset() - start, segment.getLength(), @@ -103,14 +102,14 @@ public static List mergeSegments( segment.getTaskAttemptId())); if (latestPosition - start >= readBufferSize) { dataFileSegments.add( - new DataFileSegment(path, start, (int) (latestPosition - start), bufferSegments)); + new DataFileSegment(path, start, (int) (latestPosition - start), shuffleSegments)); start = -1; } lastPosition = latestPosition; } if (start > -1) { dataFileSegments.add( - new DataFileSegment(path, start, (int) (lastPosition - start), bufferSegments)); + new DataFileSegment(path, start, (int) (lastPosition - start), shuffleSegments)); } } } diff --git a/storage/src/test/java/org/apache/uniffle/storage/HadoopShuffleHandlerTestBase.java b/storage/src/test/java/org/apache/uniffle/storage/HadoopShuffleHandlerTestBase.java index ea29a47689..5c74ed59e1 100644 --- a/storage/src/test/java/org/apache/uniffle/storage/HadoopShuffleHandlerTestBase.java +++ b/storage/src/test/java/org/apache/uniffle/storage/HadoopShuffleHandlerTestBase.java @@ -27,13 +27,12 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShufflePartitionedBlock; import org.apache.uniffle.common.util.BlockIdLayout; import org.apache.uniffle.common.util.ByteBufUtils; import org.apache.uniffle.common.util.ChecksumUtils; -import org.apache.uniffle.storage.common.FileBasedShuffleSegment; import org.apache.uniffle.storage.handler.impl.HadoopFileReader; import org.apache.uniffle.storage.handler.impl.HadoopFileWriter; import org.apache.uniffle.storage.handler.impl.HadoopShuffleWriteHandler; @@ -74,12 +73,12 @@ public static void writeTestData( long taskAttemptId, Map expectedData, Map> expectedBlocks, - Map> expectedIndexSegments, + Map> expectedIndexSegments, boolean doWrite) throws Exception { BlockIdLayout layout = BlockIdLayout.DEFAULT; List blocks = Lists.newArrayList(); - List segments = Lists.newArrayList(); + List segments = Lists.newArrayList(); for (int i = 0; i < num; i++) { byte[] buf = new byte[length]; new Random().nextBytes(buf); @@ -92,8 +91,8 @@ public static void writeTestData( expectedBlocks.put(partitionId, blocks); long offset = 0; for (ShufflePartitionedBlock spb : blocks) { - FileBasedShuffleSegment segment = - new FileBasedShuffleSegment( + ShuffleSegment segment = + new ShuffleSegment( spb.getBlockId(), offset, spb.getLength(), @@ -138,9 +137,9 @@ public static void checkData( ShuffleDataResult shuffleDataResult, Map expectedData) { byte[] buffer = shuffleDataResult.getData(); - List bufferSegments = shuffleDataResult.getBufferSegments(); + List shuffleSegments = shuffleDataResult.getBufferSegments(); - for (BufferSegment bs : bufferSegments) { + for (ShuffleSegment bs : shuffleSegments) { byte[] data = new byte[bs.getLength()]; System.arraycopy(buffer, bs.getOffset(), data, 0, bs.getLength()); assertEquals(bs.getCrc(), ChecksumUtils.getCrc32(data)); diff --git a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopClientReadHandlerTest.java b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopClientReadHandlerTest.java index fa684b840e..1f6fdabab4 100644 --- a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopClientReadHandlerTest.java +++ b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopClientReadHandlerTest.java @@ -29,11 +29,10 @@ import org.junit.jupiter.api.Test; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShuffleIndexResult; import org.apache.uniffle.storage.HadoopTestBase; -import org.apache.uniffle.storage.common.FileBasedShuffleSegment; import org.apache.uniffle.storage.util.ShuffleStorageUtils; import static org.apache.uniffle.storage.HadoopShuffleHandlerTestBase.calcExpectedSegmentNum; @@ -90,7 +89,7 @@ public static void createAndRunCases( try { ShuffleIndexResult indexResult = indexReader.readShuffleIndex(); assertEquals( - 0, indexResult.getIndexData().remaining() % FileBasedShuffleSegment.SEGMENT_SIZE); + 0, indexResult.getIndexData().remaining() % ShuffleSegment.SEGMENT_SIZE); } catch (Exception e) { fail(); } @@ -114,8 +113,8 @@ public static void createAndRunCases( ShuffleDataResult shuffleDataResult = handler.readShuffleData(); totalBlockNum += shuffleDataResult.getBufferSegments().size(); checkData(shuffleDataResult, expectedData); - for (BufferSegment bufferSegment : shuffleDataResult.getBufferSegments()) { - actualBlockIds.add(bufferSegment.getBlockId()); + for (ShuffleSegment shuffleSegment : shuffleDataResult.getBufferSegments()) { + actualBlockIds.add(shuffleSegment.getBlockId()); } } diff --git a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopFileReaderTest.java b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopFileReaderTest.java index 55c603d6f6..ce90f0a993 100644 --- a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopFileReaderTest.java +++ b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopFileReaderTest.java @@ -24,9 +24,9 @@ import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.Test; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.util.ChecksumUtils; import org.apache.uniffle.storage.HadoopTestBase; -import org.apache.uniffle.storage.common.FileBasedShuffleSegment; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -70,8 +70,8 @@ public void readDataTest() throws Exception { try (HadoopFileWriter writer = new HadoopFileWriter(fs, path, conf)) { writer.writeData(data); } - FileBasedShuffleSegment segment = - new FileBasedShuffleSegment(23, offset, length, length, 0xdeadbeef, 1); + ShuffleSegment segment = + new ShuffleSegment(23, offset, length, length, 0xdeadbeef, 1); try (HadoopFileReader reader = new HadoopFileReader(path, conf)) { byte[] actual = reader.read(segment.getOffset(), segment.getLength()); long crc22 = ChecksumUtils.getCrc32(actual); @@ -81,7 +81,7 @@ public void readDataTest() throws Exception { } assertEquals(crc11, crc22); // EOF exception is expected - segment = new FileBasedShuffleSegment(23, offset * 2, length, length, 1, 1); + segment = new ShuffleSegment(23, offset * 2, length, length, 1, 1); assertEquals(0, reader.read(segment.getOffset(), segment.getLength()).length); } } diff --git a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopFileWriterTest.java b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopFileWriterTest.java index a9d4a489f5..f0c837e079 100644 --- a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopFileWriterTest.java +++ b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopFileWriterTest.java @@ -27,8 +27,8 @@ import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.Test; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.storage.HadoopTestBase; -import org.apache.uniffle.storage.common.FileBasedShuffleSegment; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -147,7 +147,7 @@ public void writeBufferArrayTest() throws IOException { @Test public void writeSegmentTest() throws IOException { - FileBasedShuffleSegment segment = new FileBasedShuffleSegment(23, 128, 32, 32, 0xdeadbeef, 0); + ShuffleSegment segment = new ShuffleSegment(23, 128, 32, 32, 0xdeadbeef, 0); Path path = new Path(HDFS_URI, "writeSegmentTest"); try (HadoopFileWriter writer = new HadoopFileWriter(fs, path, conf)) { diff --git a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopHandlerTest.java b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopHandlerTest.java index 94c5f3c33e..0c38dfcf6f 100644 --- a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopHandlerTest.java +++ b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopHandlerTest.java @@ -30,11 +30,10 @@ import org.junit.jupiter.api.Test; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShufflePartitionedBlock; import org.apache.uniffle.storage.HadoopTestBase; -import org.apache.uniffle.storage.common.FileBasedShuffleSegment; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -56,7 +55,7 @@ public void writeTest() throws Exception { List blocks = new LinkedList<>(); List expectedBlockId = new LinkedList<>(); List expectedData = new LinkedList<>(); - List expectedIndex = new LinkedList<>(); + List expectedIndex = new LinkedList<>(); int pos = 0; for (int i = 1; i < 13; ++i) { @@ -65,7 +64,7 @@ public void writeTest() throws Exception { expectedData.add(buf); blocks.add(new ShufflePartitionedBlock(i * 8, i * 8, i, i, 0, buf)); expectedBlockId.add(Long.valueOf(i)); - expectedIndex.add(new FileBasedShuffleSegment(i, pos, i * 8, i * 8, i, 0)); + expectedIndex.add(new ShuffleSegment(i, pos, i * 8, i * 8, i, 0)); pos += i * 8; } writeHandler.write(blocks); @@ -80,7 +79,7 @@ public void writeTest() throws Exception { expectedData.add(buf); expectedBlockId.add(Long.valueOf(i)); blocksAppend.add(new ShufflePartitionedBlock(i * 8, i * 8, i, i, i, buf)); - expectedIndex.add(new FileBasedShuffleSegment(i, pos, i * 8, i * 8, i, i)); + expectedIndex.add(new ShuffleSegment(i, pos, i * 8, i * 8, i, i)); pos += i * 8; } writeHandler = new HadoopShuffleWriteHandler("appId", 1, 1, 1, basePath, "test", conf); @@ -127,9 +126,9 @@ private void compareDataAndIndex( private List readData(HadoopClientReadHandler handler, Set blockIds) throws IllegalStateException { ShuffleDataResult sdr = handler.readShuffleData(); - List bufferSegments = sdr.getBufferSegments(); + List shuffleSegments = sdr.getBufferSegments(); List result = Lists.newArrayList(); - for (BufferSegment bs : bufferSegments) { + for (ShuffleSegment bs : shuffleSegments) { byte[] data = new byte[bs.getLength()]; System.arraycopy(sdr.getData(), bs.getOffset(), data, 0, bs.getLength()); result.add(ByteBuffer.wrap(data)); diff --git a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopShuffleReadHandlerTest.java b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopShuffleReadHandlerTest.java index d1b663f1f3..ee7bb190ef 100644 --- a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopShuffleReadHandlerTest.java +++ b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopShuffleReadHandlerTest.java @@ -33,14 +33,13 @@ import org.junit.jupiter.api.Test; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.BufferSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShufflePartitionedBlock; import org.apache.uniffle.common.util.BlockIdLayout; import org.apache.uniffle.common.util.ChecksumUtils; import org.apache.uniffle.storage.HadoopShuffleHandlerTestBase; import org.apache.uniffle.storage.HadoopTestBase; -import org.apache.uniffle.storage.common.FileBasedShuffleSegment; import org.apache.uniffle.storage.util.ShuffleStorageUtils; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -81,8 +80,8 @@ public static void createAndRunCases(String clusterPathPrefix, Configuration con ShuffleDataResult shuffleDataResult = handler.readShuffleData(); totalBlockNum += shuffleDataResult.getBufferSegments().size(); HadoopShuffleHandlerTestBase.checkData(shuffleDataResult, expectedData); - for (BufferSegment bufferSegment : shuffleDataResult.getBufferSegments()) { - actualBlockIds.add(bufferSegment.getBlockId()); + for (ShuffleSegment shuffleSegment : shuffleDataResult.getBufferSegments()) { + actualBlockIds.add(shuffleSegment.getBlockId()); } } @@ -145,8 +144,8 @@ public void testDataInconsistent() throws Exception { ShuffleDataResult shuffleDataResult = handler.readShuffleData(); totalBlockNum += shuffleDataResult.getBufferSegments().size(); HadoopShuffleHandlerTestBase.checkData(shuffleDataResult, expectedData); - for (BufferSegment bufferSegment : shuffleDataResult.getBufferSegments()) { - actualBlockIds.add(bufferSegment.getBlockId()); + for (ShuffleSegment shuffleSegment : shuffleDataResult.getBufferSegments()) { + actualBlockIds.add(shuffleSegment.getBlockId()); } } @@ -208,8 +207,8 @@ public void writeIndex(List shuffleBlocks) long crc = block.getCrc(); long startOffset = indexWriter.nextOffset(); - FileBasedShuffleSegment segment = - new FileBasedShuffleSegment( + ShuffleSegment segment = + new ShuffleSegment( blockId, startOffset, block.getLength(), diff --git a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/LocalFileHandlerTestBase.java b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/LocalFileHandlerTestBase.java index 729a3a3d67..d2f67a617e 100644 --- a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/LocalFileHandlerTestBase.java +++ b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/LocalFileHandlerTestBase.java @@ -27,16 +27,15 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import org.apache.uniffle.common.BufferSegment; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShuffleDataSegment; import org.apache.uniffle.common.ShuffleIndexResult; import org.apache.uniffle.common.ShufflePartitionedBlock; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.segment.FixedSizeSegmentSplitter; import org.apache.uniffle.common.util.BlockIdLayout; import org.apache.uniffle.common.util.ByteBufUtils; import org.apache.uniffle.common.util.ChecksumUtils; -import org.apache.uniffle.storage.common.FileBasedShuffleSegment; import org.apache.uniffle.storage.handler.api.ServerReadHandler; import org.apache.uniffle.storage.handler.api.ShuffleWriteHandler; @@ -80,9 +79,9 @@ public static void validateResult( Set actualBlockIds = Sets.newHashSet(); for (ShuffleDataResult sdr : shuffleDataResults) { byte[] buffer = sdr.getData(); - List bufferSegments = sdr.getBufferSegments(); + List shuffleSegments = sdr.getBufferSegments(); - for (BufferSegment bs : bufferSegments) { + for (ShuffleSegment bs : shuffleSegments) { byte[] data = new byte[bs.getLength()]; System.arraycopy(buffer, bs.getOffset(), data, 0, bs.getLength()); assertEquals(bs.getCrc(), ChecksumUtils.getCrc32(data)); @@ -129,9 +128,9 @@ public static List readData( public static void checkData( ShuffleDataResult shuffleDataResult, Map expectedData) { byte[] buffer = shuffleDataResult.getData(); - List bufferSegments = shuffleDataResult.getBufferSegments(); + List shuffleSegments = shuffleDataResult.getBufferSegments(); - for (BufferSegment bs : bufferSegments) { + for (ShuffleSegment bs : shuffleSegments) { byte[] data = new byte[bs.getLength()]; System.arraycopy(buffer, bs.getOffset(), data, 0, bs.getLength()); assertEquals(bs.getCrc(), ChecksumUtils.getCrc32(data)); @@ -139,7 +138,7 @@ public static void checkData( } } - public static void writeIndex(ByteBuffer byteBuffer, FileBasedShuffleSegment segment) { + public static void writeIndex(ByteBuffer byteBuffer, ShuffleSegment segment) { byteBuffer.putLong(segment.getOffset()); byteBuffer.putInt(segment.getLength()); byteBuffer.putInt(segment.getUncompressLength()); diff --git a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/LocalFileServerReadHandlerTest.java b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/LocalFileServerReadHandlerTest.java index 2a55ae4de2..a727949f62 100644 --- a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/LocalFileServerReadHandlerTest.java +++ b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/LocalFileServerReadHandlerTest.java @@ -36,9 +36,9 @@ import org.apache.uniffle.client.response.RssGetShuffleIndexResponse; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShufflePartitionedBlock; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.netty.buffer.NettyManagedBuffer; import org.apache.uniffle.common.rpc.StatusCode; -import org.apache.uniffle.storage.common.FileBasedShuffleSegment; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -61,8 +61,8 @@ public void testDataInconsistent() throws Exception { shuffleBlocks -> { int offset = 0; for (ShufflePartitionedBlock block : shuffleBlocks) { - FileBasedShuffleSegment segment = - new FileBasedShuffleSegment( + ShuffleSegment segment = + new ShuffleSegment( block.getBlockId(), offset, block.getLength(), diff --git a/storage/src/test/java/org/apache/uniffle/storage/util/ShuffleStorageUtilsTest.java b/storage/src/test/java/org/apache/uniffle/storage/util/ShuffleStorageUtilsTest.java index 6f54445e57..e1fcf2c5d0 100644 --- a/storage/src/test/java/org/apache/uniffle/storage/util/ShuffleStorageUtilsTest.java +++ b/storage/src/test/java/org/apache/uniffle/storage/util/ShuffleStorageUtilsTest.java @@ -24,8 +24,7 @@ import com.google.common.collect.Sets; import org.junit.jupiter.api.Test; -import org.apache.uniffle.common.BufferSegment; -import org.apache.uniffle.storage.common.FileBasedShuffleSegment; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.storage.handler.impl.DataFileSegment; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -36,42 +35,42 @@ public class ShuffleStorageUtilsTest { @Test public void mergeSegmentsTest() { - List segments = - Lists.newArrayList(new FileBasedShuffleSegment(1, 0, 40, 0, 0, 0)); + List segments = + Lists.newArrayList(new ShuffleSegment(1, 0, 40, 0, 0, 0)); List fileSegments = ShuffleStorageUtils.mergeSegments("path", segments, 100); assertEquals(1, fileSegments.size()); for (DataFileSegment seg : fileSegments) { assertEquals(0, seg.getOffset()); assertEquals(40, seg.getLength()); assertEquals("path", seg.getPath()); - List bufferSegments = seg.getBufferSegments(); - assertEquals(1, bufferSegments.size()); - assertEquals(new BufferSegment(1, 0, 40, 0, 0, 0), bufferSegments.get(0)); + List shuffleSegments = seg.getBufferSegments(); + assertEquals(1, shuffleSegments.size()); + assertEquals(new ShuffleSegment(1, 0, 40, 0, 0, 0), shuffleSegments.get(0)); } segments = Lists.newArrayList( - new FileBasedShuffleSegment(1, 0, 40, 0, 0, 0), - new FileBasedShuffleSegment(2, 40, 40, 0, 0, 0), - new FileBasedShuffleSegment(3, 80, 20, 0, 0, 0)); + new ShuffleSegment(1, 0, 40, 0, 0, 0), + new ShuffleSegment(2, 40, 40, 0, 0, 0), + new ShuffleSegment(3, 80, 20, 0, 0, 0)); fileSegments = ShuffleStorageUtils.mergeSegments("path", segments, 100); assertEquals(1, fileSegments.size()); for (DataFileSegment seg : fileSegments) { assertEquals(0, seg.getOffset()); assertEquals(100, seg.getLength()); assertEquals("path", seg.getPath()); - List bufferSegments = seg.getBufferSegments(); - assertEquals(3, bufferSegments.size()); + List shuffleSegments = seg.getBufferSegments(); + assertEquals(3, shuffleSegments.size()); Set testedBlockIds = Sets.newHashSet(); - for (BufferSegment segment : bufferSegments) { + for (ShuffleSegment segment : shuffleSegments) { if (segment.getBlockId() == 1) { - assertTrue(segment.equals(new BufferSegment(1, 0, 40, 0, 0, 0))); + assertTrue(segment.equals(new ShuffleSegment(1, 0, 40, 0, 0, 0))); testedBlockIds.add(1L); } else if (segment.getBlockId() == 2) { - assertTrue(segment.equals(new BufferSegment(2, 40, 40, 0, 0, 0))); + assertTrue(segment.equals(new ShuffleSegment(2, 40, 40, 0, 0, 0))); testedBlockIds.add(2L); } else if (segment.getBlockId() == 3) { - assertTrue(segment.equals(new BufferSegment(3, 80, 20, 0, 0, 0))); + assertTrue(segment.equals(new ShuffleSegment(3, 80, 20, 0, 0, 0))); testedBlockIds.add(3L); } } @@ -80,10 +79,10 @@ public void mergeSegmentsTest() { segments = Lists.newArrayList( - new FileBasedShuffleSegment(1, 0, 40, 0, 0, 0), - new FileBasedShuffleSegment(2, 40, 40, 0, 0, 0), - new FileBasedShuffleSegment(3, 80, 20, 0, 0, 0), - new FileBasedShuffleSegment(4, 100, 20, 0, 0, 0)); + new ShuffleSegment(1, 0, 40, 0, 0, 0), + new ShuffleSegment(2, 40, 40, 0, 0, 0), + new ShuffleSegment(3, 80, 20, 0, 0, 0), + new ShuffleSegment(4, 100, 20, 0, 0, 0)); fileSegments = ShuffleStorageUtils.mergeSegments("path", segments, 100); assertEquals(2, fileSegments.size()); boolean tested = false; @@ -92,20 +91,20 @@ public void mergeSegmentsTest() { tested = true; assertEquals(20, seg.getLength()); assertEquals("path", seg.getPath()); - List bufferSegments = seg.getBufferSegments(); - assertEquals(1, bufferSegments.size()); - assertTrue(bufferSegments.get(0).equals(new BufferSegment(4, 0, 20, 0, 0, 0))); + List shuffleSegments = seg.getBufferSegments(); + assertEquals(1, shuffleSegments.size()); + assertTrue(shuffleSegments.get(0).equals(new ShuffleSegment(4, 0, 20, 0, 0, 0))); } } assertTrue(tested); segments = Lists.newArrayList( - new FileBasedShuffleSegment(1, 0, 40, 0, 0, 0), - new FileBasedShuffleSegment(2, 40, 40, 0, 0, 0), - new FileBasedShuffleSegment(3, 80, 20, 0, 0, 0), - new FileBasedShuffleSegment(4, 100, 20, 0, 0, 0), - new FileBasedShuffleSegment(5, 120, 100, 0, 0, 0)); + new ShuffleSegment(1, 0, 40, 0, 0, 0), + new ShuffleSegment(2, 40, 40, 0, 0, 0), + new ShuffleSegment(3, 80, 20, 0, 0, 0), + new ShuffleSegment(4, 100, 20, 0, 0, 0), + new ShuffleSegment(5, 120, 100, 0, 0, 0)); fileSegments = ShuffleStorageUtils.mergeSegments("path", segments, 100); assertEquals(2, fileSegments.size()); tested = false; @@ -114,15 +113,15 @@ public void mergeSegmentsTest() { tested = true; assertEquals(120, seg.getLength()); assertEquals("path", seg.getPath()); - List bufferSegments = seg.getBufferSegments(); - assertEquals(2, bufferSegments.size()); + List shuffleSegments = seg.getBufferSegments(); + assertEquals(2, shuffleSegments.size()); Set testedBlockIds = Sets.newHashSet(); - for (BufferSegment segment : bufferSegments) { + for (ShuffleSegment segment : shuffleSegments) { if (segment.getBlockId() == 4) { - assertTrue(segment.equals(new BufferSegment(4, 0, 20, 0, 0, 0))); + assertTrue(segment.equals(new ShuffleSegment(4, 0, 20, 0, 0, 0))); testedBlockIds.add(4L); } else if (segment.getBlockId() == 5) { - assertTrue(segment.equals(new BufferSegment(5, 20, 100, 0, 0, 0))); + assertTrue(segment.equals(new ShuffleSegment(5, 20, 100, 0, 0, 0))); testedBlockIds.add(5L); } } @@ -133,10 +132,10 @@ public void mergeSegmentsTest() { segments = Lists.newArrayList( - new FileBasedShuffleSegment(1, 10, 40, 0, 0, 0), - new FileBasedShuffleSegment(2, 80, 20, 0, 0, 0), - new FileBasedShuffleSegment(3, 500, 120, 0, 0, 0), - new FileBasedShuffleSegment(4, 700, 20, 0, 0, 0)); + new ShuffleSegment(1, 10, 40, 0, 0, 0), + new ShuffleSegment(2, 80, 20, 0, 0, 0), + new ShuffleSegment(3, 500, 120, 0, 0, 0), + new ShuffleSegment(4, 700, 20, 0, 0, 0)); fileSegments = ShuffleStorageUtils.mergeSegments("path", segments, 100); assertEquals(3, fileSegments.size()); Set expectedOffset = Sets.newHashSet(10L, 500L, 700L); @@ -147,16 +146,16 @@ public void mergeSegmentsTest() { } if (seg.getOffset() == 500) { assertEquals(120, seg.getLength()); - List bufferSegments = seg.getBufferSegments(); - assertEquals(1, bufferSegments.size()); - assertTrue(bufferSegments.get(0).equals(new BufferSegment(3, 0, 120, 0, 0, 0))); + List shuffleSegments = seg.getBufferSegments(); + assertEquals(1, shuffleSegments.size()); + assertTrue(shuffleSegments.get(0).equals(new ShuffleSegment(3, 0, 120, 0, 0, 0))); expectedOffset.remove(500L); } if (seg.getOffset() == 700) { assertEquals(20, seg.getLength()); - List bufferSegments = seg.getBufferSegments(); - assertEquals(1, bufferSegments.size()); - assertTrue(bufferSegments.get(0).equals(new BufferSegment(4, 0, 20, 0, 0, 0))); + List shuffleSegments = seg.getBufferSegments(); + assertEquals(1, shuffleSegments.size()); + assertTrue(shuffleSegments.get(0).equals(new ShuffleSegment(4, 0, 20, 0, 0, 0))); expectedOffset.remove(700L); } } @@ -164,12 +163,12 @@ public void mergeSegmentsTest() { segments = Lists.newArrayList( - new FileBasedShuffleSegment(5, 500, 120, 0, 0, 0), - new FileBasedShuffleSegment(3, 630, 10, 0, 0, 0), - new FileBasedShuffleSegment(2, 80, 20, 0, 0, 0), - new FileBasedShuffleSegment(1, 10, 40, 0, 0, 0), - new FileBasedShuffleSegment(6, 769, 20, 0, 0, 0), - new FileBasedShuffleSegment(4, 700, 20, 0, 0, 0)); + new ShuffleSegment(5, 500, 120, 0, 0, 0), + new ShuffleSegment(3, 630, 10, 0, 0, 0), + new ShuffleSegment(2, 80, 20, 0, 0, 0), + new ShuffleSegment(1, 10, 40, 0, 0, 0), + new ShuffleSegment(6, 769, 20, 0, 0, 0), + new ShuffleSegment(4, 700, 20, 0, 0, 0)); fileSegments = ShuffleStorageUtils.mergeSegments("path", segments, 100); assertEquals(4, fileSegments.size()); expectedOffset = Sets.newHashSet(10L, 500L, 630L, 700L); @@ -180,16 +179,16 @@ public void mergeSegmentsTest() { } if (seg.getOffset() == 500) { assertEquals(120, seg.getLength()); - List bufferSegments = seg.getBufferSegments(); - assertEquals(1, bufferSegments.size()); - assertTrue(bufferSegments.get(0).equals(new BufferSegment(5, 0, 120, 0, 0, 0))); + List shuffleSegments = seg.getBufferSegments(); + assertEquals(1, shuffleSegments.size()); + assertTrue(shuffleSegments.get(0).equals(new ShuffleSegment(5, 0, 120, 0, 0, 0))); expectedOffset.remove(500L); } if (seg.getOffset() == 630) { assertEquals(10, seg.getLength()); - List bufferSegments = seg.getBufferSegments(); - assertEquals(1, bufferSegments.size()); - assertTrue(bufferSegments.get(0).equals(new BufferSegment(3, 0, 10, 0, 0, 0))); + List shuffleSegments = seg.getBufferSegments(); + assertEquals(1, shuffleSegments.size()); + assertTrue(shuffleSegments.get(0).equals(new ShuffleSegment(3, 0, 10, 0, 0, 0))); expectedOffset.remove(630L); } if (seg.getOffset() == 700) { @@ -208,15 +207,15 @@ private void validResult( int anotherBlockId, int anotherOffset) { assertEquals(length, seg.getLength()); - List bufferSegments = seg.getBufferSegments(); - assertEquals(2, bufferSegments.size()); + List shuffleSegments = seg.getBufferSegments(); + assertEquals(2, shuffleSegments.size()); Set testedBlockIds = Sets.newHashSet(); - for (BufferSegment segment : bufferSegments) { + for (ShuffleSegment segment : shuffleSegments) { if (segment.getBlockId() == someBlockId) { - assertTrue(segment.equals(new BufferSegment(someBlockId, 0, someLength, 0, 0, 0))); + assertTrue(segment.equals(new ShuffleSegment(someBlockId, 0, someLength, 0, 0, 0))); testedBlockIds.add((long) someBlockId); } else if (segment.getBlockId() == anotherBlockId) { - assertTrue(segment.equals(new BufferSegment(anotherBlockId, anotherOffset, 20, 0, 0, 0))); + assertTrue(segment.equals(new ShuffleSegment(anotherBlockId, anotherOffset, 20, 0, 0, 0))); testedBlockIds.add((long) anotherBlockId); } } From 777ba2cf07ccbe2140a469cc34bf8548dffb5239 Mon Sep 17 00:00:00 2001 From: wankunde Date: Mon, 22 Jul 2024 00:22:10 +0800 Subject: [PATCH 2/3] Update code --- .../apache/uniffle/client/impl/ShuffleReadClientImpl.java | 2 +- .../test/java/org/apache/uniffle/client/TestUtils.java | 5 +++-- .../apache/uniffle/common/netty/protocol/Decoders.java | 2 +- .../apache/uniffle/common/netty/protocol/Encoders.java | 2 +- .../uniffle/common/segment/FixedSizeSegmentSplitter.java | 8 +++++--- .../uniffle/common/segment/LocalOrderSegmentSplitter.java | 5 +++-- .../uniffle/common/netty/TransportFrameDecoderTest.java | 2 +- .../uniffle/common/netty/protocol/NettyProtocolTest.java | 2 +- .../common/segment/LocalOrderSegmentSplitterTest.java | 2 +- .../uniffle/test/ShuffleServerFaultToleranceTest.java | 2 +- .../test/ShuffleServerWithLocalOfLocalOrderTest.java | 2 +- .../apache/uniffle/test/ShuffleServerWithLocalTest.java | 2 +- .../uniffle/test/ShuffleServerWithMemLocalHadoopTest.java | 2 +- .../apache/uniffle/test/ShuffleServerWithMemoryTest.java | 2 +- .../uniffle/client/impl/grpc/ShuffleServerGrpcClient.java | 2 +- .../apache/uniffle/server/ShuffleServerGrpcService.java | 2 +- .../uniffle/server/buffer/AbstractShuffleBuffer.java | 2 +- .../server/buffer/ShuffleBufferWithLinkedList.java | 2 +- .../uniffle/server/buffer/ShuffleBufferWithSkipList.java | 2 +- .../uniffle/server/netty/ShuffleServerNettyHandler.java | 5 +++-- .../apache/uniffle/server/ShuffleFlushManagerTest.java | 2 +- .../org/apache/uniffle/server/ShuffleTaskManagerTest.java | 2 +- .../server/buffer/ShuffleBufferWithLinkedListTest.java | 2 +- .../server/buffer/ShuffleBufferWithSkipListTest.java | 2 +- .../uniffle/storage/handler/api/ClientReadHandler.java | 2 +- .../storage/handler/impl/AbstractClientReadHandler.java | 2 +- .../storage/handler/impl/ComposedClientReadHandler.java | 2 +- .../uniffle/storage/handler/impl/DataFileSegment.java | 3 ++- .../uniffle/storage/handler/impl/HadoopFileWriter.java | 1 - .../storage/handler/impl/MemoryClientReadHandler.java | 2 +- .../handler/impl/MultiReplicaClientReadHandler.java | 2 +- .../uniffle/storage/HadoopShuffleHandlerTestBase.java | 2 +- .../storage/handler/impl/HadoopClientReadHandlerTest.java | 5 ++--- .../storage/handler/impl/HadoopFileReaderTest.java | 3 +-- .../uniffle/storage/handler/impl/HadoopHandlerTest.java | 2 +- .../handler/impl/HadoopShuffleReadHandlerTest.java | 2 +- .../uniffle/storage/util/ShuffleStorageUtilsTest.java | 3 +-- 37 files changed, 48 insertions(+), 46 deletions(-) diff --git a/client/src/main/java/org/apache/uniffle/client/impl/ShuffleReadClientImpl.java b/client/src/main/java/org/apache/uniffle/client/impl/ShuffleReadClientImpl.java index 02bc1a7e58..9290986646 100644 --- a/client/src/main/java/org/apache/uniffle/client/impl/ShuffleReadClientImpl.java +++ b/client/src/main/java/org/apache/uniffle/client/impl/ShuffleReadClientImpl.java @@ -35,9 +35,9 @@ import org.apache.uniffle.client.factory.ShuffleClientFactory; import org.apache.uniffle.client.response.CompressedShuffleBlock; import org.apache.uniffle.client.util.DefaultIdHelper; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataDistributionType; import org.apache.uniffle.common.ShuffleDataResult; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.config.RssClientConf; import org.apache.uniffle.common.config.RssConf; diff --git a/client/src/test/java/org/apache/uniffle/client/TestUtils.java b/client/src/test/java/org/apache/uniffle/client/TestUtils.java index 1f54fea816..6d7cf1fb06 100644 --- a/client/src/test/java/org/apache/uniffle/client/TestUtils.java +++ b/client/src/test/java/org/apache/uniffle/client/TestUtils.java @@ -24,8 +24,8 @@ import org.apache.uniffle.client.api.ShuffleReadClient; import org.apache.uniffle.client.response.CompressedShuffleBlock; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; +import org.apache.uniffle.common.ShuffleSegment; import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -76,7 +76,8 @@ public static void validateResult(Map expectedData, ShuffleDataRes } } - private static ShuffleSegment findBufferSegment(long blockId, List shuffleSegments) { + private static ShuffleSegment findBufferSegment( + long blockId, List shuffleSegments) { for (ShuffleSegment bs : shuffleSegments) { if (bs.getBlockId() == blockId) { return bs; 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 0d64ba2ba6..819a57a4ce 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 @@ -24,8 +24,8 @@ import com.google.common.collect.Maps; import io.netty.buffer.ByteBuf; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleBlockInfo; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.util.ByteBufUtils; import org.apache.uniffle.common.util.NettyUtils; diff --git a/common/src/main/java/org/apache/uniffle/common/netty/protocol/Encoders.java b/common/src/main/java/org/apache/uniffle/common/netty/protocol/Encoders.java index 54ea36b729..e9d3701644 100644 --- a/common/src/main/java/org/apache/uniffle/common/netty/protocol/Encoders.java +++ b/common/src/main/java/org/apache/uniffle/common/netty/protocol/Encoders.java @@ -21,9 +21,9 @@ import io.netty.buffer.ByteBuf; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.ShuffleBlockInfo; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.util.ByteBufUtils; diff --git a/common/src/main/java/org/apache/uniffle/common/segment/FixedSizeSegmentSplitter.java b/common/src/main/java/org/apache/uniffle/common/segment/FixedSizeSegmentSplitter.java index b14206b59b..06123d4e3d 100644 --- a/common/src/main/java/org/apache/uniffle/common/segment/FixedSizeSegmentSplitter.java +++ b/common/src/main/java/org/apache/uniffle/common/segment/FixedSizeSegmentSplitter.java @@ -25,9 +25,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataSegment; import org.apache.uniffle.common.ShuffleIndexResult; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.exception.RssException; public class FixedSizeSegmentSplitter implements SegmentSplitter { @@ -92,11 +92,13 @@ private static List transIndexDataToSegments( } shuffleSegments.add( - new ShuffleSegment(blockId, bufferOffset, length, uncompressLength, crc, taskAttemptId)); + new ShuffleSegment( + blockId, bufferOffset, length, uncompressLength, crc, taskAttemptId)); bufferOffset += length; if (bufferOffset >= readBufferSize) { - ShuffleDataSegment sds = new ShuffleDataSegment(fileOffset, bufferOffset, shuffleSegments); + ShuffleDataSegment sds = + new ShuffleDataSegment(fileOffset, bufferOffset, shuffleSegments); dataFileSegments.add(sds); shuffleSegments = Lists.newArrayList(); bufferOffset = 0; diff --git a/common/src/main/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitter.java b/common/src/main/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitter.java index 012974b996..986f24b3bb 100644 --- a/common/src/main/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitter.java +++ b/common/src/main/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitter.java @@ -27,9 +27,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataSegment; import org.apache.uniffle.common.ShuffleIndexResult; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.exception.RssException; /** @@ -122,7 +122,8 @@ public List split(ShuffleIndexResult shuffleIndexResult) { boolean conditionOfLimitedBufferSize = bufferOffset >= readBufferSize; if (conditionOfDiscontinuousBlocks || conditionOfLimitedBufferSize) { - ShuffleDataSegment sds = new ShuffleDataSegment(fileOffset, bufferOffset, shuffleSegments); + ShuffleDataSegment sds = + new ShuffleDataSegment(fileOffset, bufferOffset, shuffleSegments); dataFileSegments.add(sds); shuffleSegments = Lists.newArrayList(); bufferOffset = 0; diff --git a/common/src/test/java/org/apache/uniffle/common/netty/TransportFrameDecoderTest.java b/common/src/test/java/org/apache/uniffle/common/netty/TransportFrameDecoderTest.java index 2cb54ca5c9..c954fa05f8 100644 --- a/common/src/test/java/org/apache/uniffle/common/netty/TransportFrameDecoderTest.java +++ b/common/src/test/java/org/apache/uniffle/common/netty/TransportFrameDecoderTest.java @@ -28,8 +28,8 @@ import org.junit.jupiter.api.Test; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleBlockInfo; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.netty.buffer.NettyManagedBuffer; import org.apache.uniffle.common.netty.protocol.GetLocalShuffleDataRequest; 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 f0da66fc23..8feab7e585 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 @@ -28,8 +28,8 @@ import org.junit.jupiter.api.Test; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleBlockInfo; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.netty.buffer.NettyManagedBuffer; import org.apache.uniffle.common.rpc.StatusCode; diff --git a/common/src/test/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitterTest.java b/common/src/test/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitterTest.java index 64d77066c0..1921b58341 100644 --- a/common/src/test/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitterTest.java +++ b/common/src/test/java/org/apache/uniffle/common/segment/LocalOrderSegmentSplitterTest.java @@ -26,9 +26,9 @@ import org.junit.jupiter.params.provider.ValueSource; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataSegment; import org.apache.uniffle.common.ShuffleIndexResult; +import org.apache.uniffle.common.ShuffleSegment; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerFaultToleranceTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerFaultToleranceTest.java index a3e0096feb..9a7f40f758 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerFaultToleranceTest.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerFaultToleranceTest.java @@ -40,12 +40,12 @@ import org.apache.uniffle.client.request.RssRegisterShuffleRequest; import org.apache.uniffle.client.request.RssSendCommitRequest; import org.apache.uniffle.client.request.RssSendShuffleDataRequest; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ClientType; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.ShuffleBlockInfo; import org.apache.uniffle.common.ShuffleDataDistributionType; import org.apache.uniffle.common.ShuffleDataResult; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.rpc.ServerType; import org.apache.uniffle.common.util.ByteBufUtils; diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithLocalOfLocalOrderTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithLocalOfLocalOrderTest.java index e6d47f27cc..aaa784b788 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithLocalOfLocalOrderTest.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithLocalOfLocalOrderTest.java @@ -47,11 +47,11 @@ import org.apache.uniffle.client.request.RssSendShuffleDataRequest; import org.apache.uniffle.client.response.RssSendShuffleDataResponse; import org.apache.uniffle.client.util.DefaultIdHelper; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.RemoteStorageInfo; import org.apache.uniffle.common.ShuffleBlockInfo; import org.apache.uniffle.common.ShuffleDataResult; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.rpc.ServerType; import org.apache.uniffle.common.rpc.StatusCode; import org.apache.uniffle.common.segment.LocalOrderSegmentSplitter; diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithLocalTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithLocalTest.java index 90728735bf..67c34af6c0 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithLocalTest.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithLocalTest.java @@ -44,10 +44,10 @@ import org.apache.uniffle.client.request.RssSendCommitRequest; import org.apache.uniffle.client.request.RssSendShuffleDataRequest; import org.apache.uniffle.client.response.RssSendShuffleDataResponse; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.ShuffleBlockInfo; import org.apache.uniffle.common.ShuffleDataResult; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.rpc.ServerType; import org.apache.uniffle.common.rpc.StatusCode; import org.apache.uniffle.common.util.ChecksumUtils; diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithMemLocalHadoopTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithMemLocalHadoopTest.java index ad73ba8a83..4c56862856 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithMemLocalHadoopTest.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithMemLocalHadoopTest.java @@ -42,10 +42,10 @@ import org.apache.uniffle.client.request.RssRegisterShuffleRequest; import org.apache.uniffle.client.request.RssSendShuffleDataRequest; import org.apache.uniffle.client.response.RssSendShuffleDataResponse; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.ShuffleBlockInfo; import org.apache.uniffle.common.ShuffleDataResult; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.rpc.ServerType; import org.apache.uniffle.common.rpc.StatusCode; diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithMemoryTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithMemoryTest.java index fdb2c688aa..14bea0a465 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithMemoryTest.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/ShuffleServerWithMemoryTest.java @@ -40,10 +40,10 @@ import org.apache.uniffle.client.request.RssRegisterShuffleRequest; import org.apache.uniffle.client.request.RssSendShuffleDataRequest; import org.apache.uniffle.client.response.RssSendShuffleDataResponse; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.ShuffleBlockInfo; import org.apache.uniffle.common.ShuffleDataResult; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.rpc.ServerType; import org.apache.uniffle.common.rpc.StatusCode; diff --git a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerGrpcClient.java b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerGrpcClient.java index 86e5f0c5b1..fd85d241d0 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerGrpcClient.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerGrpcClient.java @@ -61,11 +61,11 @@ import org.apache.uniffle.client.response.RssSendShuffleDataResponse; import org.apache.uniffle.client.response.RssUnregisterShuffleByAppIdResponse; import org.apache.uniffle.client.response.RssUnregisterShuffleResponse; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.RemoteStorageInfo; import org.apache.uniffle.common.ShuffleBlockInfo; import org.apache.uniffle.common.ShuffleDataDistributionType; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.config.RssClientConf; import org.apache.uniffle.common.config.RssConf; import org.apache.uniffle.common.exception.NotRetryException; diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleServerGrpcService.java b/server/src/main/java/org/apache/uniffle/server/ShuffleServerGrpcService.java index bcb9739041..16afc4823d 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleServerGrpcService.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServerGrpcService.java @@ -37,7 +37,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.RemoteStorageInfo; import org.apache.uniffle.common.ShuffleDataDistributionType; @@ -45,6 +44,7 @@ import org.apache.uniffle.common.ShuffleIndexResult; import org.apache.uniffle.common.ShufflePartitionedBlock; import org.apache.uniffle.common.ShufflePartitionedData; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.config.RssBaseConf; import org.apache.uniffle.common.exception.FileNotFoundException; import org.apache.uniffle.common.exception.NoBufferException; diff --git a/server/src/main/java/org/apache/uniffle/server/buffer/AbstractShuffleBuffer.java b/server/src/main/java/org/apache/uniffle/server/buffer/AbstractShuffleBuffer.java index 7f1ddc309b..945248c8aa 100644 --- a/server/src/main/java/org/apache/uniffle/server/buffer/AbstractShuffleBuffer.java +++ b/server/src/main/java/org/apache/uniffle/server/buffer/AbstractShuffleBuffer.java @@ -27,10 +27,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataDistributionType; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShufflePartitionedBlock; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.util.Constants; import org.apache.uniffle.common.util.NettyUtils; import org.apache.uniffle.server.ShuffleDataFlushEvent; diff --git a/server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferWithLinkedList.java b/server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferWithLinkedList.java index 409a8f9227..cf2051b0a7 100644 --- a/server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferWithLinkedList.java +++ b/server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferWithLinkedList.java @@ -26,10 +26,10 @@ import com.google.common.collect.Lists; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataDistributionType; import org.apache.uniffle.common.ShufflePartitionedBlock; import org.apache.uniffle.common.ShufflePartitionedData; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.util.Constants; import org.apache.uniffle.common.util.JavaUtils; import org.apache.uniffle.server.ShuffleDataFlushEvent; diff --git a/server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferWithSkipList.java b/server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferWithSkipList.java index fc9c06947e..7fb12398c7 100644 --- a/server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferWithSkipList.java +++ b/server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferWithSkipList.java @@ -30,10 +30,10 @@ import com.google.common.collect.Lists; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataDistributionType; import org.apache.uniffle.common.ShufflePartitionedBlock; import org.apache.uniffle.common.ShufflePartitionedData; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.util.Constants; import org.apache.uniffle.common.util.JavaUtils; import org.apache.uniffle.server.ShuffleDataFlushEvent; 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 ada8444a45..3956e92a51 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 @@ -31,12 +31,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleBlockInfo; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShuffleIndexResult; import org.apache.uniffle.common.ShufflePartitionedBlock; import org.apache.uniffle.common.ShufflePartitionedData; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.config.RssBaseConf; import org.apache.uniffle.common.exception.FileNotFoundException; import org.apache.uniffle.common.exception.RssException; @@ -381,7 +381,8 @@ public void handleGetMemoryShuffleDataRequest( ShuffleServerMetrics.gaugeReadMemoryDataBufferSize.inc(readBufferSize); } response = - new GetMemoryShuffleDataResponse(req.getRequestId(), status, msg, shuffleSegments, data); + new GetMemoryShuffleDataResponse( + req.getRequestId(), status, msg, shuffleSegments, data); ReleaseMemoryAndRecordReadTimeListener listener = new ReleaseMemoryAndRecordReadTimeListener( start, readBufferSize, data.size(), requestInfo, req, response, client); diff --git a/server/src/test/java/org/apache/uniffle/server/ShuffleFlushManagerTest.java b/server/src/test/java/org/apache/uniffle/server/ShuffleFlushManagerTest.java index 573aeedc31..a645a68576 100644 --- a/server/src/test/java/org/apache/uniffle/server/ShuffleFlushManagerTest.java +++ b/server/src/test/java/org/apache/uniffle/server/ShuffleFlushManagerTest.java @@ -47,10 +47,10 @@ import org.junit.jupiter.api.io.TempDir; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.RemoteStorageInfo; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShufflePartitionedBlock; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.config.RssBaseConf; import org.apache.uniffle.common.util.ChecksumUtils; import org.apache.uniffle.common.util.Constants; diff --git a/server/src/test/java/org/apache/uniffle/server/ShuffleTaskManagerTest.java b/server/src/test/java/org/apache/uniffle/server/ShuffleTaskManagerTest.java index 7d1d14c4ea..54b0b3c608 100644 --- a/server/src/test/java/org/apache/uniffle/server/ShuffleTaskManagerTest.java +++ b/server/src/test/java/org/apache/uniffle/server/ShuffleTaskManagerTest.java @@ -43,12 +43,12 @@ import org.junit.jupiter.api.io.TempDir; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.PartitionRange; import org.apache.uniffle.common.RemoteStorageInfo; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShufflePartitionedBlock; import org.apache.uniffle.common.ShufflePartitionedData; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.exception.InvalidRequestException; import org.apache.uniffle.common.exception.NoBufferForHugePartitionException; import org.apache.uniffle.common.exception.NoRegisterException; diff --git a/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferWithLinkedListTest.java b/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferWithLinkedListTest.java index fae66ee75e..9ca5d6fdfb 100644 --- a/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferWithLinkedListTest.java +++ b/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferWithLinkedListTest.java @@ -24,11 +24,11 @@ import org.junit.jupiter.api.Test; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataDistributionType; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShufflePartitionedBlock; import org.apache.uniffle.common.ShufflePartitionedData; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.util.ByteBufUtils; import org.apache.uniffle.common.util.Constants; import org.apache.uniffle.server.ShuffleDataFlushEvent; diff --git a/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferWithSkipListTest.java b/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferWithSkipListTest.java index 25637717b4..c4e268e817 100644 --- a/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferWithSkipListTest.java +++ b/server/src/test/java/org/apache/uniffle/server/buffer/ShuffleBufferWithSkipListTest.java @@ -22,11 +22,11 @@ import org.junit.jupiter.api.Test; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataDistributionType; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShufflePartitionedBlock; import org.apache.uniffle.common.ShufflePartitionedData; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.util.Constants; import org.apache.uniffle.server.ShuffleDataFlushEvent; diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/api/ClientReadHandler.java b/storage/src/main/java/org/apache/uniffle/storage/handler/api/ClientReadHandler.java index 1f7bf0bdbf..bdda46843c 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/api/ClientReadHandler.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/api/ClientReadHandler.java @@ -17,8 +17,8 @@ package org.apache.uniffle.storage.handler.api; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; +import org.apache.uniffle.common.ShuffleSegment; public interface ClientReadHandler { diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/AbstractClientReadHandler.java b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/AbstractClientReadHandler.java index 7e238a3dbd..e0fb3f34da 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/AbstractClientReadHandler.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/AbstractClientReadHandler.java @@ -20,8 +20,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.storage.handler.ClientReadHandlerMetric; import org.apache.uniffle.storage.handler.api.ClientReadHandler; diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/ComposedClientReadHandler.java b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/ComposedClientReadHandler.java index 4a1b6d8407..4d65dd9af5 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/ComposedClientReadHandler.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/ComposedClientReadHandler.java @@ -29,8 +29,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.exception.RssException; import org.apache.uniffle.common.exception.RssFetchFailedException; diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/DataFileSegment.java b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/DataFileSegment.java index 57d80c01c4..22f4967a9b 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/DataFileSegment.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/DataFileSegment.java @@ -28,7 +28,8 @@ public class DataFileSegment extends FileSegment { private List shuffleSegments; - public DataFileSegment(String path, long offset, int length, List shuffleSegments) { + public DataFileSegment( + String path, long offset, int length, List shuffleSegments) { super(path, offset, length); this.shuffleSegments = shuffleSegments; } diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/HadoopFileWriter.java b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/HadoopFileWriter.java index 43b3df7753..6c0fbfca17 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/HadoopFileWriter.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/HadoopFileWriter.java @@ -27,7 +27,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IOUtils; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/MemoryClientReadHandler.java b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/MemoryClientReadHandler.java index 97da5ac4a4..531d99b8e2 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/MemoryClientReadHandler.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/MemoryClientReadHandler.java @@ -27,8 +27,8 @@ import org.apache.uniffle.client.api.ShuffleServerClient; import org.apache.uniffle.client.request.RssGetInMemoryShuffleDataRequest; import org.apache.uniffle.client.response.RssGetInMemoryShuffleDataResponse; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.exception.RssFetchFailedException; import org.apache.uniffle.common.util.Constants; diff --git a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/MultiReplicaClientReadHandler.java b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/MultiReplicaClientReadHandler.java index a5eef07628..d0a899d71d 100644 --- a/storage/src/main/java/org/apache/uniffle/storage/handler/impl/MultiReplicaClientReadHandler.java +++ b/storage/src/main/java/org/apache/uniffle/storage/handler/impl/MultiReplicaClientReadHandler.java @@ -23,8 +23,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.exception.RssException; import org.apache.uniffle.common.util.RssUtils; diff --git a/storage/src/test/java/org/apache/uniffle/storage/HadoopShuffleHandlerTestBase.java b/storage/src/test/java/org/apache/uniffle/storage/HadoopShuffleHandlerTestBase.java index 5c74ed59e1..aacb56b8d1 100644 --- a/storage/src/test/java/org/apache/uniffle/storage/HadoopShuffleHandlerTestBase.java +++ b/storage/src/test/java/org/apache/uniffle/storage/HadoopShuffleHandlerTestBase.java @@ -27,9 +27,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShufflePartitionedBlock; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.util.BlockIdLayout; import org.apache.uniffle.common.util.ByteBufUtils; import org.apache.uniffle.common.util.ChecksumUtils; diff --git a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopClientReadHandlerTest.java b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopClientReadHandlerTest.java index 1f6fdabab4..f057d3de1c 100644 --- a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopClientReadHandlerTest.java +++ b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopClientReadHandlerTest.java @@ -29,9 +29,9 @@ import org.junit.jupiter.api.Test; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShuffleIndexResult; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.storage.HadoopTestBase; import org.apache.uniffle.storage.util.ShuffleStorageUtils; @@ -88,8 +88,7 @@ public static void createAndRunCases( hadoopConf); try { ShuffleIndexResult indexResult = indexReader.readShuffleIndex(); - assertEquals( - 0, indexResult.getIndexData().remaining() % ShuffleSegment.SEGMENT_SIZE); + assertEquals(0, indexResult.getIndexData().remaining() % ShuffleSegment.SEGMENT_SIZE); } catch (Exception e) { fail(); } diff --git a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopFileReaderTest.java b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopFileReaderTest.java index ce90f0a993..73bfaf69bc 100644 --- a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopFileReaderTest.java +++ b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopFileReaderTest.java @@ -70,8 +70,7 @@ public void readDataTest() throws Exception { try (HadoopFileWriter writer = new HadoopFileWriter(fs, path, conf)) { writer.writeData(data); } - ShuffleSegment segment = - new ShuffleSegment(23, offset, length, length, 0xdeadbeef, 1); + ShuffleSegment segment = new ShuffleSegment(23, offset, length, length, 0xdeadbeef, 1); try (HadoopFileReader reader = new HadoopFileReader(path, conf)) { byte[] actual = reader.read(segment.getOffset(), segment.getLength()); long crc22 = ChecksumUtils.getCrc32(actual); diff --git a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopHandlerTest.java b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopHandlerTest.java index 0c38dfcf6f..91ccf1a29b 100644 --- a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopHandlerTest.java +++ b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopHandlerTest.java @@ -30,9 +30,9 @@ import org.junit.jupiter.api.Test; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShufflePartitionedBlock; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.storage.HadoopTestBase; import static org.junit.jupiter.api.Assertions.assertTrue; diff --git a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopShuffleReadHandlerTest.java b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopShuffleReadHandlerTest.java index ee7bb190ef..8d83dab8ca 100644 --- a/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopShuffleReadHandlerTest.java +++ b/storage/src/test/java/org/apache/uniffle/storage/handler/impl/HadoopShuffleReadHandlerTest.java @@ -33,9 +33,9 @@ import org.junit.jupiter.api.Test; import org.roaringbitmap.longlong.Roaring64NavigableMap; -import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.ShuffleDataResult; import org.apache.uniffle.common.ShufflePartitionedBlock; +import org.apache.uniffle.common.ShuffleSegment; import org.apache.uniffle.common.util.BlockIdLayout; import org.apache.uniffle.common.util.ChecksumUtils; import org.apache.uniffle.storage.HadoopShuffleHandlerTestBase; diff --git a/storage/src/test/java/org/apache/uniffle/storage/util/ShuffleStorageUtilsTest.java b/storage/src/test/java/org/apache/uniffle/storage/util/ShuffleStorageUtilsTest.java index e1fcf2c5d0..d958dda72a 100644 --- a/storage/src/test/java/org/apache/uniffle/storage/util/ShuffleStorageUtilsTest.java +++ b/storage/src/test/java/org/apache/uniffle/storage/util/ShuffleStorageUtilsTest.java @@ -35,8 +35,7 @@ public class ShuffleStorageUtilsTest { @Test public void mergeSegmentsTest() { - List segments = - Lists.newArrayList(new ShuffleSegment(1, 0, 40, 0, 0, 0)); + List segments = Lists.newArrayList(new ShuffleSegment(1, 0, 40, 0, 0, 0)); List fileSegments = ShuffleStorageUtils.mergeSegments("path", segments, 100); assertEquals(1, fileSegments.size()); for (DataFileSegment seg : fileSegments) { From 0687ceebb9568379495a868cd8c8a7f740ce1329 Mon Sep 17 00:00:00 2001 From: wankunde Date: Mon, 22 Jul 2024 09:39:19 +0800 Subject: [PATCH 3/3] Update --- .../main/java/org/apache/uniffle/common/ShuffleSegment.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/common/src/main/java/org/apache/uniffle/common/ShuffleSegment.java b/common/src/main/java/org/apache/uniffle/common/ShuffleSegment.java index 4b62dce188..aad4db423e 100644 --- a/common/src/main/java/org/apache/uniffle/common/ShuffleSegment.java +++ b/common/src/main/java/org/apache/uniffle/common/ShuffleSegment.java @@ -21,9 +21,6 @@ import org.apache.uniffle.common.exception.RssException; -/** - * The metadata for a segment, it can be a memory segment or a file segment. - */ public class ShuffleSegment implements java.io.Serializable, Comparable { public static final int SEGMENT_SIZE = 4 * Long.BYTES + 2 * Integer.BYTES;