Skip to content
Merged
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -154,10 +154,30 @@ public void add(int batchIndex, ShuffleDataResult shuffleDataResult) {
}

public DecompressedShuffleBlock get(int batchIndex, int segmentIndex) {
// guardedly safe to remove the previous batches if exist since the upstream will fetch the
// segments in order
for (int i = 0; i < batchIndex; i++) {
ConcurrentHashMap<Integer, DecompressedShuffleBlock> prevBlocks = tasks.remove(i);
if (prevBlocks != null) {
for (DecompressedShuffleBlock block : prevBlocks.values()) {
segmentPermits.ifPresent(x -> x.release());
}
}
}

ConcurrentHashMap<Integer, DecompressedShuffleBlock> blocks = tasks.get(batchIndex);
if (blocks == null) {
return null;
}

// guardedly safe to remove the previous segments if exist since the upstream will fetch the
// segments in order
for (int i = 0; i < segmentIndex; i++) {
if (blocks.remove(i) != null) {
segmentPermits.ifPresent(x -> x.release());
}
}

DecompressedShuffleBlock block = blocks.remove(segmentIndex);
// simplify the memory statistic logic here, just decrease the memory used when the block is
// fetched, this is effective due to the upstream will use single-thread to get and release the
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -314,8 +314,16 @@ public ShuffleBlock readShuffleBlockData() {
// mark block as processed
processedBlockIds.add(bs.getBlockId());
pendingBlockIds.removeLong(bs.getBlockId());
// update the segment index to skip the unnecessary block in overlapping decompression mode
segmentIndex += 1;

// update the segment index to skip the unnecessary block in overlapping decompression mode.
// In overlapping decompression mode, decompression tasks for the whole batch have already
// been submitted. If we skip a segment without removing the corresponding handler, the
// backpressure permits may never be released, which can block subsequent decompression.
if (decompressionWorker != null) {
decompressionWorker.get(batchIndex - 1, segmentIndex++);
} else {
segmentIndex += 1;
}
}

if (bs != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import static org.apache.uniffle.common.config.RssClientConf.COMPRESSION_TYPE;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;

public class DecompressionWorkerTest {

Expand Down Expand Up @@ -66,7 +67,9 @@ public void testBackpressure() throws Exception {
}
Thread.sleep(10);
worker.get(0, maxSegments).getByteBuffer();
assertEquals(1024 * maxSegments, worker.getPeekMemoryUsed());
// Peak memory is a runtime metric and may include one additional segment due to thread timing.
assertTrue(worker.getPeekMemoryUsed() <= 1024L * (maxSegments + 1));
assertTrue(worker.getPeekMemoryUsed() >= 1024L * maxSegments);
assertEquals(maxSegments, worker.getAvailablePermits());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@
import org.apache.uniffle.storage.handler.impl.HadoopShuffleWriteHandler;
import org.apache.uniffle.storage.util.StorageType;

import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertNull;
Expand Down Expand Up @@ -769,6 +770,47 @@ public void readTest16(Supplier<ShuffleClientFactory.ReadClientBuilder> builderS
readClient.close();
}

@ParameterizedTest
@MethodSource("clientBuilderProvider")
public void readTestSkipBlocksWithBackpressureDoesNotHang(
Supplier<ShuffleClientFactory.ReadClientBuilder> builderSupplier) throws Exception {
// This test is meaningful only when overlapping decompression is enabled.
// For non-overlapping mode, it should still pass and act as a regression guard.
String basePath = uniq(HDFS_URI + "clientReadTestSkipBlocksWithBackpressureDoesNotHang");
HadoopShuffleWriteHandler writeHandler =
new HadoopShuffleWriteHandler("appId", 0, 1, 1, basePath, ssi1.getId(), conf);

Map<Long, byte[]> expectedData = Maps.newHashMap();
Roaring64NavigableMap blockIdBitmap = Roaring64NavigableMap.bitmapOf();

// Write skipped blocks first to increase the chance of exhausting permits if permits are not
// released when skipping.
writeTestData(writeHandler, 20, 30, 1, 2, Maps.newHashMap(), blockIdBitmap);
writeTestData(writeHandler, 5, 30, 1, 0, expectedData, blockIdBitmap);

RssConf rssConf = new RssConf();
// Provide required base configs to avoid reader treating this as "prod mode" with empty values.
rssConf.set(RssClientConf.RSS_STORAGE_TYPE, StorageType.HDFS.name());
rssConf.setInteger(RssClientConf.RSS_READ_OVERLAPPING_DECOMPRESSION_FETCH_SECONDS_THRESHOLD, 1);
rssConf.setInteger(RssClientConf.RSS_READ_OVERLAPPING_DECOMPRESSION_MAX_CONCURRENT_SEGMENTS, 1);

// Expect only taskAttemptId=0 blocks; taskAttemptId=2 blocks will be skipped by the reader.
Roaring64NavigableMap taskIdBitmap = Roaring64NavigableMap.bitmapOf(0);
ShuffleReadClientImpl readClient =
builderSupplier
.get()
.partitionId(1)
.basePath(basePath)
.blockIdBitmap(blockIdBitmap)
.taskIdBitmap(taskIdBitmap)
.rssConf(rssConf)
.build();

assertDoesNotThrow(() -> TestUtils.validateResult(readClient, expectedData));
readClient.checkProcessedBlockIds();
readClient.close();
}

private void writeTestData(
HadoopShuffleWriteHandler writeHandler,
int num,
Expand Down
Loading