Skip to content
Open
Show file tree
Hide file tree
Changes from all 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 @@ -21,6 +21,7 @@
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicLong;

import com.google.common.annotations.VisibleForTesting;
Expand Down Expand Up @@ -64,13 +65,15 @@ public class ShuffleFlushManager {
// appId -> shuffleId -> committed shuffle blockIds
private Map<String, Map<Integer, Roaring64NavigableMap>> committedBlockIds =
JavaUtils.newConcurrentMap();
private List<Integer> shuffleIdsWithWriteError = new CopyOnWriteArrayList<>();
private final int retryMax;

private final StorageManager storageManager;
private final long pendingEventTimeoutSec;
private FlushEventHandler eventHandler;
private final boolean isAuditLogEnabled;


public ShuffleFlushManager(
ShuffleServerConf shuffleServerConf,
ShuffleServer shuffleServer,
Expand Down Expand Up @@ -168,12 +171,20 @@ public void processFlushEvent(ShuffleDataFlushEvent event) throws Exception {
storageDataReplica,
user,
maxConcurrencyPerPartitionToWrite);
ShuffleWriteHandler handler = storage.getOrCreateWriteHandler(request);
long startTime = System.currentTimeMillis();
boolean writeSuccess = storageManager.write(storage, handler, event);
if (!writeSuccess) {
throw new EventRetryException();

long startTime = 0L;
try {
ShuffleWriteHandler handler = storage.getOrCreateWriteHandler(request);
startTime = System.currentTimeMillis();
boolean writeSuccess = storageManager.write(storage, handler, event);
if (!writeSuccess) {
shuffleIdsWithWriteError.add(event.getShuffleId());
Copy link
Contributor

@rickyma rickyma Jul 22, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think this is thread-safe.

It could be overwritten by other threads which may be successful.

You need to reconsider concurrency problems through this PR.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Apologies for the delayed response, but here I used CopyOnWriteArrayList for implementing shuffleIdsWithWriteError, do you still think thread safety is a concern here, or do you think the other parts of the code are not thread safe?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please help to review this.Thx.@maobaolong

}
} catch (Exception e) {
LOG.error("storageManager write error.", e);
shuffleIdsWithWriteError.add(event.getShuffleId());
}

long endTime = System.currentTimeMillis();

// update some metrics for shuffle task
Expand Down Expand Up @@ -243,7 +254,11 @@ private void updateCommittedBlockIds(
}
}

public Roaring64NavigableMap getCommittedBlockIds(String appId, Integer shuffleId) {
public Roaring64NavigableMap getCommittedBlockIds(String appId, Integer shuffleId) throws EventDiscardException {
if (shuffleIdsWithWriteError.contains(shuffleId)) {
throw new EventDiscardException();
}

Map<Integer, Roaring64NavigableMap> shuffleIdToBlockIds = committedBlockIds.get(appId);
if (shuffleIdToBlockIds == null) {
LOG.warn("Unexpected value when getCommittedBlockIds for appId[" + appId + "]");
Expand Down Expand Up @@ -299,4 +314,9 @@ public ShuffleDataDistributionType getDataDistributionType(String appId) {
public FlushEventHandler getEventHandler() {
return eventHandler;
}

@VisibleForTesting
public void setShuffleIdsWithWriteError(int shuffleId) {
shuffleIdsWithWriteError.add(shuffleId);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.uniffle.server.flush.EventDiscardException;
import org.awaitility.Awaitility;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
Expand Down Expand Up @@ -75,6 +76,7 @@
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;

Expand Down Expand Up @@ -1048,6 +1050,53 @@ public void checkAndClearLeakShuffleDataTest(@TempDir File tempDir) throws Excep
assertTrue(hiddenFile.exists());
}

@Test
public void testCommitShuffleFailOnWriteFail() throws Exception {
String confFile = ClassLoader.getSystemResource("server.conf").getFile();
ShuffleServerConf conf = new ShuffleServerConf(confFile);
final String remoteStorage = HDFS_URI + "rss/test";
final String appId = "testAppId";
final int shuffleId = 1;
conf.set(ShuffleServerConf.SERVER_BUFFER_CAPACITY, 128L);
conf.set(ShuffleServerConf.SERVER_MEMORY_SHUFFLE_HIGHWATERMARK_PERCENTAGE, 50.0);
conf.set(ShuffleServerConf.SERVER_MEMORY_SHUFFLE_LOWWATERMARK_PERCENTAGE, 0.0);
conf.setString(ShuffleServerConf.RSS_STORAGE_TYPE.key(), StorageType.HDFS.name());
conf.set(ShuffleServerConf.RSS_TEST_MODE_ENABLE, true);
conf.set(ShuffleServerConf.SERVER_COMMIT_TIMEOUT, 10000L);
conf.set(ShuffleServerConf.SERVER_PRE_ALLOCATION_EXPIRED, 3000L);
conf.set(ShuffleServerConf.HEALTH_CHECK_ENABLE, false);
shuffleServer = new ShuffleServer(conf);
ShuffleTaskManager shuffleTaskManager = shuffleServer.getShuffleTaskManager();
shuffleTaskManager.registerShuffle(
appId,
shuffleId,
Lists.newArrayList(new PartitionRange(1, 1)),
new RemoteStorageInfo(remoteStorage),
StringUtils.EMPTY);
shuffleTaskManager.registerShuffle(
appId,
shuffleId,
Lists.newArrayList(new PartitionRange(2, 2)),
new RemoteStorageInfo(remoteStorage),
StringUtils.EMPTY);
final List<ShufflePartitionedBlock> expectedBlocks1 = Lists.newArrayList();
final List<ShufflePartitionedBlock> expectedBlocks2 = Lists.newArrayList();
final Map<Long, PreAllocatedBufferInfo> bufferIds = shuffleTaskManager.getRequireBufferIds();

shuffleTaskManager.requireBuffer(10);
shuffleTaskManager.requireBuffer(10);
shuffleTaskManager.requireBuffer(10);
assertEquals(3, bufferIds.size());
// required buffer should be clear if it doesn't receive data after timeout
Thread.sleep(6000);
assertEquals(0, bufferIds.size());

ShuffleFlushManager shuffleFlushManager = shuffleServer.getShuffleFlushManager();
shuffleFlushManager.setShuffleIdsWithWriteError(shuffleId);

assertThrows(EventDiscardException.class, () -> shuffleTaskManager.commitShuffle(appId, shuffleId));
}

private Set<String> getAppIdsOnDisk(LocalStorageManager localStorageManager) {
Set<String> appIdsOnDisk = new HashSet<>();

Expand Down