Skip to content
Merged
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
@@ -0,0 +1,30 @@
/*
* 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.common.exception;

public class NotRetryException extends RssException {

public NotRetryException(String message) {
super(message);
}

public NotRetryException(String message, Throwable e) {
super(message, e);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.uniffle.common.exception.NotRetryException;

public class RetryUtils {
private static final Logger LOG = LoggerFactory.getLogger(RetryUtils.class);

Expand All @@ -48,7 +50,8 @@ public static <T> T retry(RetryCmd<T> cmd, RetryCallBack callBack, long interval
return ret;
} catch (Throwable t) {
retry++;
if ((exceptionClasses != null && !isInstanceOf(exceptionClasses, t)) || retry >= retryTimes) {
if ((exceptionClasses != null && !isInstanceOf(exceptionClasses, t)) || retry >= retryTimes
|| t instanceof NotRetryException) {
throw t;
} else {
LOG.info("Retry due to Throwable, " + t.getClass().getName() + " " + t.getMessage());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import java.util.concurrent.atomic.AtomicInteger;

import com.google.common.collect.Sets;
import org.apache.uniffle.common.exception.NotRetryException;
import org.junit.jupiter.api.Test;

import org.apache.uniffle.common.exception.RssException;
Expand Down Expand Up @@ -67,5 +68,16 @@ public void testRetry() {
// ignore
}
assertEquals(tryTimes.get(), 1);

tryTimes.set(0);
try {
RetryUtils.retry(() -> {
tryTimes.incrementAndGet();
throw new NotRetryException("");
}, 10, maxTryTime);
} catch (Throwable throwable) {
// ignore
}
assertEquals(tryTimes.get(), 1);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.common.io.Files;
import com.google.protobuf.ByteString;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Disabled;
Expand Down Expand Up @@ -56,6 +57,7 @@
import org.apache.uniffle.common.config.RssBaseConf;
import org.apache.uniffle.common.util.Constants;
import org.apache.uniffle.coordinator.CoordinatorConf;
import org.apache.uniffle.proto.RssProtos;
import org.apache.uniffle.server.ShuffleDataFlushEvent;
import org.apache.uniffle.server.ShuffleServerConf;
import org.apache.uniffle.server.ShuffleServerGrpcMetrics;
Expand Down Expand Up @@ -393,6 +395,47 @@ public void sendDataWithoutRegisterTest() throws Exception {
assertEquals(0, shuffleServers.get(0).getPreAllocatedMemory());
}


@Test
public void sendDataWithoutRequirePreAllocation() throws Exception {
String appId = "sendDataWithoutRequirePreAllocation";
List<ShuffleBlockInfo> blockInfos = Lists.newArrayList(new ShuffleBlockInfo(0, 0, 0, 100, 0,
new byte[]{}, Lists.newArrayList(), 0, 100, 0));
Map<Integer, List<ShuffleBlockInfo>> partitionToBlocks = Maps.newHashMap();
partitionToBlocks.put(0, blockInfos);
Map<Integer, Map<Integer, List<ShuffleBlockInfo>>> shuffleToBlocks = Maps.newHashMap();
shuffleToBlocks.put(0, partitionToBlocks);
for (Map.Entry<Integer, Map<Integer, List<ShuffleBlockInfo>>> stb : shuffleToBlocks.entrySet()) {
List<RssProtos.ShuffleData> shuffleData = Lists.newArrayList();
for (Map.Entry<Integer, List<ShuffleBlockInfo>> ptb : stb.getValue().entrySet()) {
List<RssProtos.ShuffleBlock> shuffleBlocks = Lists.newArrayList();
for (ShuffleBlockInfo sbi : ptb.getValue()) {
shuffleBlocks.add(RssProtos.ShuffleBlock.newBuilder().setBlockId(sbi.getBlockId())
.setCrc(sbi.getCrc())
.setLength(sbi.getLength())
.setTaskAttemptId(sbi.getTaskAttemptId())
.setUncompressLength(sbi.getUncompressLength())
.setData(ByteString.copyFrom(sbi.getData()))
.build());
}
shuffleData.add(RssProtos.ShuffleData.newBuilder().setPartitionId(ptb.getKey())
.addAllBlock(shuffleBlocks)
.build());
}

RssProtos.SendShuffleDataRequest rpcRequest = RssProtos.SendShuffleDataRequest.newBuilder()
.setAppId(appId)
.setShuffleId(0)
.setRequireBufferId(10000)
.addAllShuffleData(shuffleData)
.build();
RssProtos.SendShuffleDataResponse response =
shuffleServerClient.getBlockingStub().sendShuffleData(rpcRequest);
assertTrue(RssProtos.StatusCode.INTERNAL_ERROR.equals(response.getStatus()));
assertTrue(response.getRetMsg().contains("Can't find requireBufferId[10000]"));
}
}

@Test
public void multipleShuffleResultTest() throws Exception {
Set<Long> expectedBlockIds = Sets.newConcurrentHashSet();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,9 @@
import org.apache.uniffle.common.PartitionRange;
import org.apache.uniffle.common.RemoteStorageInfo;
import org.apache.uniffle.common.ShuffleBlockInfo;
import org.apache.uniffle.common.exception.NotRetryException;
import org.apache.uniffle.common.exception.RssException;
import org.apache.uniffle.common.util.RetryUtils;
import org.apache.uniffle.proto.RssProtos.AppHeartBeatRequest;
import org.apache.uniffle.proto.RssProtos.AppHeartBeatResponse;
import org.apache.uniffle.proto.RssProtos.FinishShuffleRequest;
Expand Down Expand Up @@ -109,7 +111,7 @@ public ShuffleServerGrpcClient(String host, int port, int maxRetryAttempts, bool
blockingStub = ShuffleServerGrpc.newBlockingStub(channel);
}

private ShuffleServerBlockingStub getBlockingStub() {
public ShuffleServerBlockingStub getBlockingStub() {
return blockingStub.withDeadlineAfter(rpcTimeout, TimeUnit.MILLISECONDS);
}

Expand Down Expand Up @@ -253,29 +255,40 @@ public RssSendShuffleDataResponse sendShuffleData(RssSendShuffleDataRequest requ
.build());
}

long requireId = requirePreAllocation(size, request.getRetryMax(), request.getRetryIntervalMax());
if (requireId != FAILED_REQUIRE_ID) {
SendShuffleDataRequest rpcRequest = SendShuffleDataRequest.newBuilder()
.setAppId(appId)
.setShuffleId(stb.getKey())
.setRequireBufferId(requireId)
.addAllShuffleData(shuffleData)
.build();
long start = System.currentTimeMillis();
SendShuffleDataResponse response = doSendData(rpcRequest);
LOG.info("Do sendShuffleData to {}:{} rpc cost:" + (System.currentTimeMillis() - start)
+ " ms for " + size + " bytes with " + blockNum + " blocks", host, port);

if (response.getStatus() != StatusCode.SUCCESS) {
String msg = "Can't send shuffle data with " + blockNum
+ " blocks to " + host + ":" + port
+ ", statusCode=" + response.getStatus()
+ ", errorMsg:" + response.getRetMsg();
LOG.warn(msg);
isSuccessful = false;
break;
}
} else {
final int allocateSize = size;
final int finalBlockNum = blockNum;
try {
RetryUtils.retry(() -> {
long requireId = requirePreAllocation(allocateSize, request.getRetryMax(), request.getRetryIntervalMax());
if (requireId == FAILED_REQUIRE_ID) {
throw new RssException(String.format(
"requirePreAllocation failed! size[%s], host[%s], port[%s]", allocateSize, host, port));
}
SendShuffleDataRequest rpcRequest = SendShuffleDataRequest.newBuilder()
.setAppId(appId)
.setShuffleId(stb.getKey())
.setRequireBufferId(requireId)
.addAllShuffleData(shuffleData)
.build();
long start = System.currentTimeMillis();
SendShuffleDataResponse response = getBlockingStub().sendShuffleData(rpcRequest);
LOG.info("Do sendShuffleData to {}:{} rpc cost:" + (System.currentTimeMillis() - start)
+ " ms for " + allocateSize + " bytes with " + finalBlockNum + " blocks", host, port);
if (response.getStatus() != StatusCode.SUCCESS) {
String msg = "Can't send shuffle data with " + finalBlockNum
+ " blocks to " + host + ":" + port
+ ", statusCode=" + response.getStatus()
+ ", errorMsg:" + response.getRetMsg();
if (response.getStatus() == StatusCode.NO_REGISTER) {
throw new NotRetryException(msg);
} else {
throw new RssException(msg);
}
}
return response;
}, request.getRetryIntervalMax(), maxRetryAttempts);
} catch (Throwable throwable) {
LOG.warn(throwable.getMessage());
isSuccessful = false;
break;
}
Expand All @@ -290,21 +303,6 @@ public RssSendShuffleDataResponse sendShuffleData(RssSendShuffleDataRequest requ
return response;
}

private SendShuffleDataResponse doSendData(SendShuffleDataRequest rpcRequest) {
int retryNum = 0;
while (retryNum < maxRetryAttempts) {
try {
SendShuffleDataResponse response = getBlockingStub().sendShuffleData(rpcRequest);
return response;
} catch (Exception e) {
retryNum++;
LOG.warn("Send data to host[" + host + "], port[" + port
+ "] failed, try again, retryNum[" + retryNum + "]", e);
}
}
throw new RssException("Send data to host[" + host + "], port[" + port + "] failed");
}

@Override
public RssSendCommitResponse sendCommit(RssSendCommitRequest request) {
ShuffleCommitResponse rpcResponse = doSendCommit(request.getAppId(), request.getShuffleId());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -150,8 +150,18 @@ public void sendShuffleData(SendShuffleDataRequest req,
ShuffleServerMetrics.counterTotalReceivedDataSize.inc(requireSize);
boolean isPreAllocated = shuffleServer.getShuffleTaskManager().isPreAllocated(requireBufferId);
if (!isPreAllocated) {
LOG.warn("Can't find requireBufferId[" + requireBufferId + "] for appId[" + appId
+ "], shuffleId[" + shuffleId + "]");
String errorMsg = "Can't find requireBufferId[" + requireBufferId + "] for appId[" + appId
+ "], shuffleId[" + shuffleId + "]";
LOG.warn(errorMsg);
responseMessage = errorMsg;
reply = SendShuffleDataResponse
.newBuilder()
.setStatus(valueOf(StatusCode.INTERNAL_ERROR))
.setRetMsg(responseMessage)
.build();
responseObserver.onNext(reply);
responseObserver.onCompleted();
return;
}
final long start = System.currentTimeMillis();
List<ShufflePartitionedData> shufflePartitionedData = toPartitionedData(req);
Expand Down