Skip to content
Merged
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 @@ -24,10 +24,12 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;

Expand Down Expand Up @@ -93,6 +95,8 @@ public class RssShuffleWriter<K, V, C> extends ShuffleWriter<K, V> {

protected final ShuffleWriteMetrics shuffleWriteMetrics;

private final BlockingQueue<Object> finishEventQueue = new LinkedBlockingQueue<>();

// Only for tests
@VisibleForTesting
public RssShuffleWriter(
Expand Down Expand Up @@ -293,24 +297,47 @@ protected List<CompletableFuture<Long>> postBlockEvent(
List<ShuffleBlockInfo> shuffleBlockInfoList) {
List<CompletableFuture<Long>> futures = new ArrayList<>();
for (AddBlockEvent event : bufferManager.buildBlockEvents(shuffleBlockInfoList)) {
event.addCallback(
() -> {
boolean ret = finishEventQueue.add(new Object());
if (!ret) {
LOG.error("Add event " + event + " to finishEventQueue fail");
}
});
futures.add(shuffleManager.sendData(event));
}
return futures;
}

@VisibleForTesting
protected void checkBlockSendResult(Set<Long> blockIds) {
long start = System.currentTimeMillis();
while (true) {
checkIfBlocksFailed();
Set<Long> successBlockIds = shuffleManager.getSuccessBlockIds(taskId);
blockIds.removeAll(successBlockIds);
if (blockIds.isEmpty()) {
break;
boolean interrupted = false;
Copy link
Member

Choose a reason for hiding this comment

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

It looks complex for current implement.

And I think the sendTimeout is wrong, which is valid for all everytime queue.poll

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We use remainingMs instead of sendCheckTImeout.

Copy link
Member

Choose a reason for hiding this comment

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

My fault. +1


try {
long remainingMs = sendCheckTimeout;
long end = System.currentTimeMillis() + remainingMs;

while (true) {
try {
finishEventQueue.clear();
checkIfBlocksFailed();
Set<Long> successBlockIds = shuffleManager.getSuccessBlockIds(taskId);
blockIds.removeAll(successBlockIds);
if (blockIds.isEmpty()) {
break;
}
if (finishEventQueue.isEmpty()) {
remainingMs = Math.max(end - System.currentTimeMillis(), 0);
Object event = finishEventQueue.poll(remainingMs, TimeUnit.MILLISECONDS);
if (event == null) {
break;
}
}
} catch (InterruptedException e) {
interrupted = true;
}
}
LOG.info("Wait " + blockIds.size() + " blocks sent to shuffle server");
Uninterruptibles.sleepUninterruptibly(sendCheckInterval, TimeUnit.MILLISECONDS);
if (System.currentTimeMillis() - start > sendCheckTimeout) {
if (!blockIds.isEmpty()) {
String errorMsg =
"Timeout: Task["
+ taskId
Expand All @@ -322,6 +349,10 @@ protected void checkBlockSendResult(Set<Long> blockIds) {
LOG.error(errorMsg);
throw new RssException(errorMsg);
}
} finally {
if (interrupted) {
Thread.currentThread().interrupt();
}
}
}

Expand Down