diff --git a/client-mr/core/src/main/java/org/apache/hadoop/mapreduce/v2/app/RssMRAppMaster.java b/client-mr/core/src/main/java/org/apache/hadoop/mapreduce/v2/app/RssMRAppMaster.java index 06973d0797..43d0476e7d 100644 --- a/client-mr/core/src/main/java/org/apache/hadoop/mapreduce/v2/app/RssMRAppMaster.java +++ b/client-mr/core/src/main/java/org/apache/hadoop/mapreduce/v2/app/RssMRAppMaster.java @@ -285,7 +285,6 @@ public Thread newThread(Runnable r) { ShuffleDataDistributionType.NORMAL, RssMRConfig.toRssConf(conf) .get(MAX_CONCURRENCY_PER_PARTITION_TO_WRITE), - 0, remoteMergeEnable ? MergeContext.newBuilder() .setKeyClass(conf.getMapOutputKeyClass().getName()) diff --git a/client-mr/core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/FetcherTest.java b/client-mr/core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/FetcherTest.java index 82a98a84ef..871f9e58f9 100644 --- a/client-mr/core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/FetcherTest.java +++ b/client-mr/core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/FetcherTest.java @@ -507,7 +507,6 @@ public void registerShuffle( RemoteStorageInfo storageType, ShuffleDataDistributionType distributionType, int maxConcurrencyPerPartitionToWrite, - int stageAttemptNumber, RssProtos.MergeContext mergeContext, Map properties) {} @@ -548,9 +547,6 @@ public ShuffleAssignmentsInfo getShuffleAssignments( int assignmentShuffleServerNumber, int estimateTaskConcurrency, Set faultyServerIds, - int stageId, - int stageAttemptNumber, - boolean reassign, long retryIntervalMs, int retryTimes) { return null; diff --git a/client-mr/core/src/test/java/org/apache/hadoop/mapred/SortWriteBufferManagerTest.java b/client-mr/core/src/test/resources/SortWriteBufferManagerTest.java similarity index 99% rename from client-mr/core/src/test/java/org/apache/hadoop/mapred/SortWriteBufferManagerTest.java rename to client-mr/core/src/test/resources/SortWriteBufferManagerTest.java index 77a008c1c0..09decc9181 100644 --- a/client-mr/core/src/test/java/org/apache/hadoop/mapred/SortWriteBufferManagerTest.java +++ b/client-mr/core/src/test/resources/SortWriteBufferManagerTest.java @@ -721,7 +721,6 @@ public void registerShuffle( RemoteStorageInfo remoteStorage, ShuffleDataDistributionType distributionType, int maxConcurrencyPerPartitionToWrite, - int stageAttemptNumber, RssProtos.MergeContext mergeContext, Map properties) {} @@ -781,9 +780,6 @@ public ShuffleAssignmentsInfo getShuffleAssignments( int assignmentShuffleServerNumber, int estimateTaskConcurrency, Set faultyServerIds, - int stageId, - int stageAttemptNumber, - boolean reassign, long retryIntervalMs, int retryTimes) { return null; diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java index 4a7f653db6..22d162b474 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkConfig.java @@ -46,22 +46,6 @@ public class RssSparkConfig { .withDeprecatedKeys(RssClientConfig.RSS_RESUBMIT_STAGE) .withDescription("Whether to enable the resubmit stage for fetch/write failure"); - public static final ConfigOption RSS_RESUBMIT_STAGE_WITH_FETCH_FAILURE_ENABLED = - ConfigOptions.key("rss.stageRetry.fetchFailureEnabled") - .booleanType() - .defaultValue(false) - .withFallbackKeys(RSS_RESUBMIT_STAGE_ENABLED.key(), RssClientConfig.RSS_RESUBMIT_STAGE) - .withDescription( - "If set to true, the stage retry mechanism will be enabled when a fetch failure occurs."); - - public static final ConfigOption RSS_RESUBMIT_STAGE_WITH_WRITE_FAILURE_ENABLED = - ConfigOptions.key("rss.stageRetry.writeFailureEnabled") - .booleanType() - .defaultValue(false) - .withFallbackKeys(RSS_RESUBMIT_STAGE_ENABLED.key(), RssClientConfig.RSS_RESUBMIT_STAGE) - .withDescription( - "If set to true, the stage retry mechanism will be enabled when a write failure occurs."); - public static final ConfigOption RSS_BLOCK_ID_SELF_MANAGEMENT_ENABLED = ConfigOptions.key("rss.blockId.selfManagementEnabled") .booleanType() diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java index c37a936235..fc09014e1d 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/RssSparkShuffleUtils.java @@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.spark.SparkConf; import org.apache.spark.SparkContext; +import org.apache.spark.TaskContext; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.deploy.SparkHadoopUtil; import org.apache.spark.shuffle.handle.SimpleShuffleHandleInfo; @@ -54,7 +55,7 @@ import org.apache.uniffle.common.exception.RssFetchFailedException; import org.apache.uniffle.common.util.Constants; -import static org.apache.spark.shuffle.RssSparkConfig.RSS_RESUBMIT_STAGE_WITH_FETCH_FAILURE_ENABLED; +import static org.apache.spark.shuffle.RssSparkConfig.RSS_RESUBMIT_STAGE_ENABLED; public class RssSparkShuffleUtils { @@ -366,17 +367,21 @@ public static RssException reportRssFetchFailedException( SparkConf sparkConf, String appId, int shuffleId, + int uniffleShuffleId, int stageAttemptId, + int stageAttemptNumber, Set failedPartitions) { RssConf rssConf = RssSparkConfig.toRssConf(sparkConf); - if (rssConf.getBoolean(RSS_RESUBMIT_STAGE_WITH_FETCH_FAILURE_ENABLED) + if (rssConf.getBoolean(RSS_RESUBMIT_STAGE_ENABLED) && RssSparkShuffleUtils.isStageResubmitSupported()) { for (int partitionId : failedPartitions) { RssReportShuffleFetchFailureRequest req = new RssReportShuffleFetchFailureRequest( appId, shuffleId, + uniffleShuffleId, stageAttemptId, + stageAttemptNumber, partitionId, rssFetchFailedException.getMessage()); RssReportShuffleFetchFailureResponse response = @@ -404,4 +409,8 @@ public static boolean isSparkUIEnabled(SparkConf conf) { } return false; } + + public static String getAppShuffleIdentifier(int appShuffleId, TaskContext context) { + return appShuffleId + "-" + context.stageId() + "-" + context.stageAttemptNumber(); + } } diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/ShuffleHandleInfoManager.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/ShuffleHandleInfoManager.java index cc3d3b4ce9..aa04dcc038 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/ShuffleHandleInfoManager.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/ShuffleHandleInfoManager.java @@ -21,18 +21,18 @@ import java.io.IOException; import java.util.Map; -import org.apache.spark.shuffle.handle.ShuffleHandleInfo; +import org.apache.spark.shuffle.handle.MutableShuffleHandleInfo; import org.apache.uniffle.common.util.JavaUtils; public class ShuffleHandleInfoManager implements Closeable { - private Map shuffleIdToShuffleHandleInfo; + private Map shuffleIdToShuffleHandleInfo; public ShuffleHandleInfoManager() { this.shuffleIdToShuffleHandleInfo = JavaUtils.newConcurrentMap(); } - public ShuffleHandleInfo get(int shuffleId) { + public MutableShuffleHandleInfo get(int shuffleId) { return shuffleIdToShuffleHandleInfo.get(shuffleId); } @@ -40,7 +40,7 @@ public void remove(int shuffleId) { shuffleIdToShuffleHandleInfo.remove(shuffleId); } - public void register(int shuffleId, ShuffleHandleInfo handle) { + public void register(int shuffleId, MutableShuffleHandleInfo handle) { shuffleIdToShuffleHandleInfo.put(shuffleId, handle); } diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/handle/StageAttemptShuffleHandleInfo.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/handle/StageAttemptShuffleHandleInfo.java deleted file mode 100644 index 4ca7b8153b..0000000000 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/handle/StageAttemptShuffleHandleInfo.java +++ /dev/null @@ -1,144 +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.spark.shuffle.handle; - -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import com.google.common.collect.Lists; -import org.apache.spark.shuffle.handle.split.PartitionSplitInfo; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.uniffle.client.PartitionDataReplicaRequirementTracking; -import org.apache.uniffle.common.RemoteStorageInfo; -import org.apache.uniffle.common.ShuffleServerInfo; -import org.apache.uniffle.proto.RssProtos; - -public class StageAttemptShuffleHandleInfo extends ShuffleHandleInfoBase { - private static final long serialVersionUID = 0L; - private static final Logger LOGGER = LoggerFactory.getLogger(StageAttemptShuffleHandleInfo.class); - - private ShuffleHandleInfo current; - /** When Stage retry occurs, record the Shuffle Server of the previous Stage. */ - private LinkedList historyHandles; - - public StageAttemptShuffleHandleInfo( - int shuffleId, RemoteStorageInfo remoteStorage, ShuffleHandleInfo shuffleServerInfo) { - super(shuffleId, remoteStorage); - this.current = shuffleServerInfo; - this.historyHandles = Lists.newLinkedList(); - } - - public StageAttemptShuffleHandleInfo( - int shuffleId, - RemoteStorageInfo remoteStorage, - ShuffleHandleInfo currentShuffleServerInfo, - LinkedList historyHandles) { - super(shuffleId, remoteStorage); - this.current = currentShuffleServerInfo; - this.historyHandles = historyHandles; - } - - @Override - public Set getServers() { - return current.getServers(); - } - - @Override - public Map> getAvailablePartitionServersForWriter() { - return current.getAvailablePartitionServersForWriter(); - } - - @Override - public Map> getAllPartitionServersForReader() { - return current.getAllPartitionServersForReader(); - } - - @Override - public PartitionDataReplicaRequirementTracking createPartitionReplicaTracking() { - return current.createPartitionReplicaTracking(); - } - - @Override - public PartitionSplitInfo getPartitionSplitInfo(int partitionId) { - return current.getPartitionSplitInfo(partitionId); - } - - /** - * When a Stage retry occurs, replace the current shuffleHandleInfo and record the historical - * shuffleHandleInfo. - */ - public void replaceCurrentShuffleHandleInfo(ShuffleHandleInfo shuffleHandleInfo) { - this.historyHandles.add(current); - this.current = shuffleHandleInfo; - } - - public ShuffleHandleInfo getCurrent() { - return current; - } - - public LinkedList getHistoryHandles() { - return historyHandles; - } - - public static RssProtos.StageAttemptShuffleHandleInfo toProto( - StageAttemptShuffleHandleInfo handleInfo) { - LinkedList mutableShuffleHandleInfoLinkedList = - Lists.newLinkedList(); - RssProtos.MutableShuffleHandleInfo currentMutableShuffleHandleInfo = - MutableShuffleHandleInfo.toProto((MutableShuffleHandleInfo) handleInfo.getCurrent()); - for (ShuffleHandleInfo historyHandle : handleInfo.getHistoryHandles()) { - mutableShuffleHandleInfoLinkedList.add( - MutableShuffleHandleInfo.toProto((MutableShuffleHandleInfo) historyHandle)); - } - RssProtos.StageAttemptShuffleHandleInfo handleProto = - RssProtos.StageAttemptShuffleHandleInfo.newBuilder() - .setCurrentMutableShuffleHandleInfo(currentMutableShuffleHandleInfo) - .addAllHistoryMutableShuffleHandleInfo(mutableShuffleHandleInfoLinkedList) - .build(); - return handleProto; - } - - public static StageAttemptShuffleHandleInfo fromProto( - RssProtos.StageAttemptShuffleHandleInfo handleProto) { - if (handleProto == null) { - return null; - } - - MutableShuffleHandleInfo mutableShuffleHandleInfo = - MutableShuffleHandleInfo.fromProto(handleProto.getCurrentMutableShuffleHandleInfo()); - List historyMutableShuffleHandleInfoList = - handleProto.getHistoryMutableShuffleHandleInfoList(); - LinkedList historyHandles = Lists.newLinkedList(); - for (RssProtos.MutableShuffleHandleInfo shuffleHandleInfo : - historyMutableShuffleHandleInfoList) { - historyHandles.add(MutableShuffleHandleInfo.fromProto(shuffleHandleInfo)); - } - - StageAttemptShuffleHandleInfo stageAttemptShuffleHandleInfo = - new StageAttemptShuffleHandleInfo( - mutableShuffleHandleInfo.shuffleId, - mutableShuffleHandleInfo.remoteStorage, - mutableShuffleHandleInfo, - historyHandles); - return stageAttemptShuffleHandleInfo; - } -} diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/reader/RssFetchFailedIterator.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/reader/RssFetchFailedIterator.java index 1bc61dc746..6d938ffcd7 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/reader/RssFetchFailedIterator.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/reader/RssFetchFailedIterator.java @@ -48,8 +48,10 @@ private RssFetchFailedIterator(Builder builder, Iterator> iterato public static class Builder { private String appId; private int shuffleId; + private int uniffleShuffleId; private int partitionId; private int stageAttemptId; + private int stageAttemptNumber; private Supplier managerClientSupplier; private Builder() {} @@ -64,6 +66,11 @@ Builder shuffleId(int shuffleId) { return this; } + Builder uniffleShuffleId(int uniffleShuffleId) { + this.uniffleShuffleId = uniffleShuffleId; + return this; + } + Builder partitionId(int partitionId) { this.partitionId = partitionId; return this; @@ -74,6 +81,11 @@ Builder stageAttemptId(int stageAttemptId) { return this; } + Builder stageAttemptNumber(int stageAttemptNumber) { + this.stageAttemptNumber = stageAttemptNumber; + return this; + } + Builder managerClientSupplier(Supplier managerClientSupplier) { this.managerClientSupplier = managerClientSupplier; return this; @@ -95,7 +107,9 @@ private RssException generateFetchFailedIfNecessary(RssFetchFailedException e) { new RssReportShuffleFetchFailureRequest( builder.appId, builder.shuffleId, + builder.uniffleShuffleId, builder.stageAttemptId, + builder.stageAttemptNumber, builder.partitionId, e.getMessage()); RssReportShuffleFetchFailureResponse response = client.reportShuffleFetchFailure(req); diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/AddBlockEvent.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/AddBlockEvent.java index 24df3e6d1d..7212a9037c 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/AddBlockEvent.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/AddBlockEvent.java @@ -25,22 +25,19 @@ public class AddBlockEvent { private String taskId; - private int stageAttemptNumber; private List shuffleDataInfoList; private List processedCallbackChain; private WriteBufferManager bufferManager; public AddBlockEvent(String taskId, List shuffleDataInfoList) { - this(taskId, 0, shuffleDataInfoList, null); + this(taskId, shuffleDataInfoList, null); } public AddBlockEvent( String taskId, - int stageAttemptNumber, List shuffleDataInfoList, WriteBufferManager writeBufferManager) { this.taskId = taskId; - this.stageAttemptNumber = stageAttemptNumber; this.shuffleDataInfoList = shuffleDataInfoList; this.processedCallbackChain = new ArrayList<>(); this.bufferManager = writeBufferManager; @@ -55,10 +52,6 @@ public String getTaskId() { return taskId; } - public int getStageAttemptNumber() { - return stageAttemptNumber; - } - public List getShuffleDataInfoList() { return shuffleDataInfoList; } diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/DataPusher.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/DataPusher.java index f145836544..dfd5ecdb4f 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/DataPusher.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/DataPusher.java @@ -93,10 +93,7 @@ public CompletableFuture send(AddBlockEvent event) { try { result = shuffleWriteClient.sendShuffleData( - rssAppId, - event.getStageAttemptNumber(), - shuffleBlockInfoList, - () -> !isValidTask(taskId)); + rssAppId, shuffleBlockInfoList, () -> !isValidTask(taskId)); putBlockId(taskToSuccessBlockIds, taskId, result.getSuccessBlockIds()); putFailedBlockSendTracker( taskToFailedBlockSendTracker, taskId, result.getFailedBlockSendTracker()); diff --git a/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/WriteBufferManager.java b/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/WriteBufferManager.java index 169bb20b17..eacd768303 100644 --- a/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/WriteBufferManager.java +++ b/client-spark/common/src/main/java/org/apache/spark/shuffle/writer/WriteBufferManager.java @@ -105,7 +105,6 @@ public class WriteBufferManager extends MemoryConsumer { private BlockIdLayout blockIdLayout; private double bufferSpillRatio; private Function> partitionAssignmentRetrieveFunc; - private int stageAttemptNumber; private ShuffleServerPushCostTracker shuffleServerPushCostTracker; public WriteBufferManager( @@ -127,8 +126,7 @@ public WriteBufferManager( taskMemoryManager, shuffleWriteMetrics, rssConf, - null, - 0); + null); } public WriteBufferManager( @@ -142,32 +140,6 @@ public WriteBufferManager( RssConf rssConf, Function, List>> spillFunc, Function> partitionAssignmentRetrieveFunc) { - this( - shuffleId, - taskId, - taskAttemptId, - bufferManagerOptions, - serializer, - taskMemoryManager, - shuffleWriteMetrics, - rssConf, - spillFunc, - partitionAssignmentRetrieveFunc, - 0); - } - - public WriteBufferManager( - int shuffleId, - String taskId, - long taskAttemptId, - BufferManagerOptions bufferManagerOptions, - Serializer serializer, - TaskMemoryManager taskMemoryManager, - ShuffleWriteMetrics shuffleWriteMetrics, - RssConf rssConf, - Function, List>> spillFunc, - Function> partitionAssignmentRetrieveFunc, - int stageAttemptNumber) { super(taskMemoryManager, taskMemoryManager.pageSizeBytes(), MemoryMode.ON_HEAP); this.bufferSize = bufferManagerOptions.getBufferSize(); this.spillSize = bufferManagerOptions.getBufferSpillThreshold(); @@ -201,7 +173,6 @@ public WriteBufferManager( this.bufferSpillRatio = rssConf.get(RssSparkConfig.RSS_MEMORY_SPILL_RATIO); this.blockIdLayout = BlockIdLayout.from(rssConf); this.partitionAssignmentRetrieveFunc = partitionAssignmentRetrieveFunc; - this.stageAttemptNumber = stageAttemptNumber; this.shuffleServerPushCostTracker = new ShuffleServerPushCostTracker(); } @@ -215,8 +186,7 @@ public WriteBufferManager( TaskMemoryManager taskMemoryManager, ShuffleWriteMetrics shuffleWriteMetrics, RssConf rssConf, - Function, List>> spillFunc, - int stageAttemptNumber) { + Function, List>> spillFunc) { this( shuffleId, taskId, @@ -227,8 +197,7 @@ public WriteBufferManager( shuffleWriteMetrics, rssConf, spillFunc, - partitionId -> partitionToServers.get(partitionId), - stageAttemptNumber); + partitionId -> partitionToServers.get(partitionId)); } /** add serialized columnar data directly when integrate with gluten */ @@ -531,7 +500,7 @@ public List buildBlockEvents(List shuffleBlockI + totalSize + " bytes"); } - events.add(new AddBlockEvent(taskId, stageAttemptNumber, shuffleBlockInfosPerEvent, this)); + events.add(new AddBlockEvent(taskId, shuffleBlockInfosPerEvent, this)); shuffleBlockInfosPerEvent = Lists.newArrayList(); totalSize = 0; } @@ -546,7 +515,7 @@ public List buildBlockEvents(List shuffleBlockI + " bytes"); } // Use final temporary variables for closures - events.add(new AddBlockEvent(taskId, stageAttemptNumber, shuffleBlockInfosPerEvent, this)); + events.add(new AddBlockEvent(taskId, shuffleBlockInfosPerEvent, this)); } return events; } diff --git a/client-spark/common/src/main/java/org/apache/uniffle/shuffle/BlockIdSelfManagedShuffleWriteClient.java b/client-spark/common/src/main/java/org/apache/uniffle/shuffle/BlockIdSelfManagedShuffleWriteClient.java index a407442f34..93aa3f0fc0 100644 --- a/client-spark/common/src/main/java/org/apache/uniffle/shuffle/BlockIdSelfManagedShuffleWriteClient.java +++ b/client-spark/common/src/main/java/org/apache/uniffle/shuffle/BlockIdSelfManagedShuffleWriteClient.java @@ -77,31 +77,6 @@ public void reportShuffleResult( managerClientSupplier.get().reportShuffleResult(request); } - @Override - public void reportShuffleResult( - Map>> serverToPartitionToBlockIds, - String appId, - int shuffleId, - long taskAttemptId, - int bitmapNum, - Set reportFailureServers, - boolean enableWriteFailureRetry) { - Map> partitionToBlockIds = new HashMap<>(); - for (Map> k : serverToPartitionToBlockIds.values()) { - for (Map.Entry> entry : k.entrySet()) { - int partitionId = entry.getKey(); - partitionToBlockIds - .computeIfAbsent(partitionId, x -> new ArrayList<>()) - .addAll(entry.getValue()); - } - } - - RssReportShuffleResultRequest request = - new RssReportShuffleResultRequest( - appId, shuffleId, taskAttemptId, partitionToBlockIds, bitmapNum); - managerClientSupplier.get().reportShuffleResult(request); - } - @Override public Roaring64NavigableMap getShuffleResult( String clientType, diff --git a/client-spark/common/src/main/java/org/apache/uniffle/shuffle/ShuffleIdMappingManager.java b/client-spark/common/src/main/java/org/apache/uniffle/shuffle/ShuffleIdMappingManager.java new file mode 100644 index 0000000000..7d3d0b54a1 --- /dev/null +++ b/client-spark/common/src/main/java/org/apache/uniffle/shuffle/ShuffleIdMappingManager.java @@ -0,0 +1,113 @@ +/* + * 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.shuffle; + +import java.util.Collection; +import java.util.Comparator; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.uniffle.common.util.JavaUtils; + +/** + * We establish a Spark shuffleId and a stage attemptId attemptNumber, and the correspondence + * between them and the Uniffle shuffleId. The Uniffle shuffleId is incrementing forever. For + * example: + * + *

| spark shuffleId | shuffleId_stageId_attemptNumber | uniffle shuffleId | + * + *

| 0 | 0_0_0 | 0 | + * + *

| 0 | 0_0_1 | 1 | + * + *

| 0 | 0_0_2 | 2 | + * + *

| 0 | 0_0_3 | 3 | + * + *

| 1 | 1_1_0 | 4 | + * + *

| 1 | 1_1_1 | 5 | + * + *

... + */ +public class ShuffleIdMappingManager { + // Generate a new ShuffleID. + private AtomicInteger shuffleIdGenerator; + // appShuffleId -> app_stageid_attemptnumber -> newShuffleId. + private Map> shuffleIdMapping; + // Map the relationship between shuffleId and Determinate. + private Map shuffleDeterminateMap; + + public ShuffleIdMappingManager() { + shuffleIdGenerator = new AtomicInteger(-1); + shuffleIdMapping = JavaUtils.newConcurrentMap(); + shuffleDeterminateMap = JavaUtils.newConcurrentMap(); + } + + /** + * Create the shuffleId of uniffle based on the ShuffleID of Spark. When registerShuffle is being + * performed, the default number of attempts by our stage is 0. + * + * @param shuffleId + * @return + */ + public int createUniffleShuffleId(int shuffleId, String appShuffleIdentifier) { + Map appShuffleIdentifier2NewShuffleIdMap = shuffleIdMapping.get(shuffleId); + if (appShuffleIdentifier2NewShuffleIdMap == null) { + appShuffleIdentifier2NewShuffleIdMap = JavaUtils.newConcurrentMap(); + appShuffleIdentifier2NewShuffleIdMap.computeIfAbsent( + appShuffleIdentifier, k -> shuffleIdGenerator.incrementAndGet()); + shuffleIdMapping.put(shuffleId, appShuffleIdentifier2NewShuffleIdMap); + return appShuffleIdentifier2NewShuffleIdMap.get(appShuffleIdentifier); + } else { + return appShuffleIdentifier2NewShuffleIdMap.computeIfAbsent( + appShuffleIdentifier, k -> shuffleIdGenerator.incrementAndGet()); + } + } + + public boolean hasUniffleShuffleId(int shuffleId, String appShuffleIdentifier) { + if (shuffleIdMapping.isEmpty() + || shuffleIdMapping.get(shuffleId) == null + || shuffleIdMapping.get(shuffleId).get(appShuffleIdentifier) == null) { + return false; + } + return true; + } + + public int getUniffleShuffleId(int shuffleId, String appShuffleIdentifier) { + return shuffleIdMapping.get(shuffleId).get(appShuffleIdentifier); + } + + public int getUniffleShuffleIdForRead(int shuffleId) { + Map stringIntegerMap = shuffleIdMapping.get(shuffleId); + Collection values = stringIntegerMap.values(); + values.stream().findFirst().get(); + return shuffleIdMapping.get(shuffleId).values().stream() + .sorted(Comparator.reverseOrder()) + .findFirst() + .get(); + } + + public void recordShuffleIdDeterminate(int shuffleId, boolean isDeterminate) { + shuffleDeterminateMap.put(shuffleId, isDeterminate); + } + + public boolean getShuffleIdDeterminate(int shuffleId) { + return shuffleDeterminateMap.get(shuffleId) && (shuffleIdMapping.get(shuffleId) != null); + } +} diff --git a/client-spark/common/src/main/java/org/apache/uniffle/shuffle/manager/RssShuffleManagerBase.java b/client-spark/common/src/main/java/org/apache/uniffle/shuffle/manager/RssShuffleManagerBase.java index cc3d903648..1d33a0cf66 100644 --- a/client-spark/common/src/main/java/org/apache/uniffle/shuffle/manager/RssShuffleManagerBase.java +++ b/client-spark/common/src/main/java/org/apache/uniffle/shuffle/manager/RssShuffleManagerBase.java @@ -44,7 +44,6 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; import org.apache.commons.collections4.CollectionUtils; -import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; import org.apache.spark.MapOutputTracker; @@ -52,6 +51,7 @@ import org.apache.spark.SparkConf; import org.apache.spark.SparkEnv; import org.apache.spark.SparkException; +import org.apache.spark.TaskContext; import org.apache.spark.shuffle.RssShuffleHandle; import org.apache.spark.shuffle.RssSparkConfig; import org.apache.spark.shuffle.RssSparkShuffleUtils; @@ -62,7 +62,6 @@ import org.apache.spark.shuffle.handle.MutableShuffleHandleInfo; import org.apache.spark.shuffle.handle.ShuffleHandleInfo; import org.apache.spark.shuffle.handle.SimpleShuffleHandleInfo; -import org.apache.spark.shuffle.handle.StageAttemptShuffleHandleInfo; import org.apache.spark.shuffle.writer.AddBlockEvent; import org.apache.spark.shuffle.writer.DataPusher; import org.slf4j.Logger; @@ -74,11 +73,11 @@ import org.apache.uniffle.client.factory.ShuffleManagerClientFactory; import org.apache.uniffle.client.impl.FailedBlockSendTracker; import org.apache.uniffle.client.impl.grpc.CoordinatorGrpcRetryableClient; +import org.apache.uniffle.client.request.RssAppUniffleShuffleIdRequest; import org.apache.uniffle.client.request.RssFetchClientConfRequest; import org.apache.uniffle.client.request.RssPartitionToShuffleServerRequest; import org.apache.uniffle.client.response.RssFetchClientConfResponse; import org.apache.uniffle.client.response.RssReassignOnBlockSendFailureResponse; -import org.apache.uniffle.client.response.RssReassignOnStageRetryResponse; import org.apache.uniffle.client.util.ClientUtils; import org.apache.uniffle.common.ClientType; import org.apache.uniffle.common.PartitionRange; @@ -102,11 +101,10 @@ import org.apache.uniffle.common.util.RssUtils; import org.apache.uniffle.common.util.ThreadUtils; import org.apache.uniffle.shuffle.BlockIdManager; +import org.apache.uniffle.shuffle.ShuffleIdMappingManager; import static org.apache.spark.shuffle.RssSparkConfig.RSS_BLOCK_ID_SELF_MANAGEMENT_ENABLED; import static org.apache.spark.shuffle.RssSparkConfig.RSS_PARTITION_REASSIGN_MAX_REASSIGNMENT_SERVER_NUM; -import static org.apache.spark.shuffle.RssSparkConfig.RSS_RESUBMIT_STAGE_WITH_FETCH_FAILURE_ENABLED; -import static org.apache.spark.shuffle.RssSparkConfig.RSS_RESUBMIT_STAGE_WITH_WRITE_FAILURE_ENABLED; import static org.apache.spark.shuffle.RssSparkShuffleUtils.isSparkUIEnabled; import static org.apache.uniffle.common.config.RssBaseConf.RPC_SERVER_PORT; import static org.apache.uniffle.common.config.RssClientConf.HADOOP_CONFIG_KEY_PREFIX; @@ -144,8 +142,6 @@ public abstract class RssShuffleManagerBase implements RssShuffleManagerInterfac protected Map shuffleIdToNumMapTasks; protected Supplier managerClientSupplier; protected boolean rssStageRetryEnabled; - protected boolean rssStageRetryForWriteFailureEnabled; - protected boolean rssStageRetryForFetchFailureEnabled; /** * Mapping between ShuffleId and ShuffleServer list. ShuffleServer list is dynamically allocated. * ShuffleServer is not obtained from RssShuffleHandle, but from this mapping. @@ -153,6 +149,7 @@ public abstract class RssShuffleManagerBase implements RssShuffleManagerInterfac protected ShuffleHandleInfoManager shuffleHandleInfoManager; protected RssStageResubmitManager rssStageResubmitManager; + protected ShuffleIdMappingManager shuffleIdMappingManager; protected int partitionReassignMaxServerNum; protected boolean blockIdSelfManagedEnabled; protected boolean partitionReassignEnabled; @@ -251,25 +248,7 @@ public RssShuffleManagerBase(SparkConf conf, boolean isDriver) { this.shuffleIdToPartitionNum = JavaUtils.newConcurrentMap(); this.shuffleIdToNumMapTasks = JavaUtils.newConcurrentMap(); - // stage retry for write/fetch failure - rssStageRetryForFetchFailureEnabled = - rssConf.get(RSS_RESUBMIT_STAGE_WITH_FETCH_FAILURE_ENABLED); - rssStageRetryForWriteFailureEnabled = - rssConf.get(RSS_RESUBMIT_STAGE_WITH_WRITE_FAILURE_ENABLED); - if (rssStageRetryForFetchFailureEnabled || rssStageRetryForWriteFailureEnabled) { - rssStageRetryEnabled = true; - List logTips = new ArrayList<>(); - if (rssStageRetryForWriteFailureEnabled) { - logTips.add("write"); - } - if (rssStageRetryForWriteFailureEnabled) { - logTips.add("fetch"); - } - LOG.info( - "Activate the stage retry mechanism that will resubmit stage on {} failure", - StringUtils.join(logTips, "/")); - } - + this.rssStageRetryEnabled = rssConf.get(RssSparkConfig.RSS_RESUBMIT_STAGE_ENABLED); this.partitionReassignEnabled = rssConf.get(RssClientConf.RSS_CLIENT_REASSIGN_ENABLED); // The feature of partition reassign is exclusive with multiple replicas and stage retry. if (partitionReassignEnabled) { @@ -340,6 +319,7 @@ public RssShuffleManagerBase(SparkConf conf, boolean isDriver) { rssConf.get(RSS_PARTITION_REASSIGN_MAX_REASSIGNMENT_SERVER_NUM); this.shuffleHandleInfoManager = new ShuffleHandleInfoManager(); this.rssStageResubmitManager = new RssStageResubmitManager(); + this.shuffleIdMappingManager = new ShuffleIdMappingManager(); } @VisibleForTesting @@ -390,6 +370,7 @@ protected RssShuffleManagerBase( rssConf.get(RSS_PARTITION_REASSIGN_MAX_REASSIGNMENT_SERVER_NUM); this.shuffleHandleInfoManager = new ShuffleHandleInfoManager(); this.rssStageResubmitManager = new RssStageResubmitManager(); + this.shuffleIdMappingManager = new ShuffleIdMappingManager(); } public BlockIdManager getBlockIdManager() { @@ -856,65 +837,69 @@ protected static RemoteStorageInfo getDefaultRemoteStorageInfo(SparkConf sparkCo } public ShuffleHandleInfo getShuffleHandleInfo( - int stageAttemptId, - int stageAttemptNumber, - RssShuffleHandle rssHandle, - boolean isWritePhase) { - int shuffleId = rssHandle.getShuffleId(); - if (shuffleManagerRpcServiceEnabled && rssStageRetryForWriteFailureEnabled) { - // In Stage Retry mode, Get the ShuffleServer list from the Driver based on the shuffleId. - return getRemoteShuffleHandleInfoWithStageRetry( - stageAttemptId, stageAttemptNumber, shuffleId, isWritePhase); - } else if (shuffleManagerRpcServiceEnabled && partitionReassignEnabled) { - // In partition block Retry mode, Get the ShuffleServer list from the Driver based on the - // shuffleId. - return getRemoteShuffleHandleInfoWithBlockRetry( - stageAttemptId, stageAttemptNumber, shuffleId, isWritePhase); + int shuffleId, RssShuffleHandle rssHandle) { + if (rssStageRetryEnabled || partitionReassignEnabled) { + RssPartitionToShuffleServerRequest rssPartitionToShuffleServerRequest = + new RssPartitionToShuffleServerRequest(shuffleId); + RssReassignOnBlockSendFailureResponse rpcPartitionToShufflerServer = + getOrCreateShuffleManagerClientSupplier() + .get() + .getPartitionToShufflerServer(rssPartitionToShuffleServerRequest); + MutableShuffleHandleInfo mutableShuffleHandleInfo = + MutableShuffleHandleInfo.fromProto(rpcPartitionToShufflerServer.getHandle()); + return mutableShuffleHandleInfo; } else { - return new SimpleShuffleHandleInfo( - shuffleId, rssHandle.getPartitionToServers(), rssHandle.getRemoteStorage()); + SimpleShuffleHandleInfo simpleShuffleHandleInfo = + new SimpleShuffleHandleInfo( + shuffleId, rssHandle.getPartitionToServers(), getRemoteStorageInfo()); + return simpleShuffleHandleInfo; } } - /** - * In Stage Retry mode, obtain the Shuffle Server list from the Driver based on shuffleId. - * - * @param shuffleId shuffleId - * @return ShuffleHandleInfo - */ - protected synchronized StageAttemptShuffleHandleInfo getRemoteShuffleHandleInfoWithStageRetry( - int stageAttemptId, int stageAttemptNumber, int shuffleId, boolean isWritePhase) { - RssPartitionToShuffleServerRequest rssPartitionToShuffleServerRequest = - new RssPartitionToShuffleServerRequest( - stageAttemptId, stageAttemptNumber, shuffleId, isWritePhase); - RssReassignOnStageRetryResponse rpcPartitionToShufflerServer = - getOrCreateShuffleManagerClientSupplier() - .get() - .getPartitionToShufflerServerWithStageRetry(rssPartitionToShuffleServerRequest); - StageAttemptShuffleHandleInfo shuffleHandleInfo = - StageAttemptShuffleHandleInfo.fromProto( - rpcPartitionToShufflerServer.getShuffleHandleInfoProto()); - return shuffleHandleInfo; + protected int getUniffleShuffleId(int shuffleId, TaskContext context, boolean isWriter) { + String appShuffleIdentifier = RssSparkShuffleUtils.getAppShuffleIdentifier(shuffleId, context); + return getOrCreateShuffleManagerClientSupplier() + .get() + .getUniffleShuffleId( + new RssAppUniffleShuffleIdRequest(shuffleId, appShuffleIdentifier, isWriter)) + .getGeneratorShuffleId(); } - /** - * In Block Retry mode, obtain the Shuffle Server list from the Driver based on shuffleId. - * - * @param shuffleId shuffleId - * @return ShuffleHandleInfo - */ - protected synchronized MutableShuffleHandleInfo getRemoteShuffleHandleInfoWithBlockRetry( - int stageAttemptId, int stageAttemptNumber, int shuffleId, boolean isWritePhase) { - RssPartitionToShuffleServerRequest rssPartitionToShuffleServerRequest = - new RssPartitionToShuffleServerRequest( - stageAttemptId, stageAttemptNumber, shuffleId, isWritePhase); - RssReassignOnBlockSendFailureResponse rpcPartitionToShufflerServer = - getOrCreateShuffleManagerClientSupplier() - .get() - .getPartitionToShufflerServerWithBlockRetry(rssPartitionToShuffleServerRequest); - MutableShuffleHandleInfo shuffleHandleInfo = - MutableShuffleHandleInfo.fromProto(rpcPartitionToShufflerServer.getHandle()); - return shuffleHandleInfo; + @Override + public synchronized int getUniffleShuffleId( + int shuffleId, String appShuffleIdentifier, boolean isWriter) { + Integer newShuffleId; + if (isWriter) { + if (shuffleIdMappingManager.hasUniffleShuffleId(shuffleId, appShuffleIdentifier)) { + newShuffleId = shuffleIdMappingManager.getUniffleShuffleId(shuffleId, appShuffleIdentifier); + } else { + if (shuffleIdMappingManager.getShuffleIdDeterminate(shuffleId)) { + newShuffleId = shuffleIdMappingManager.getUniffleShuffleIdForRead(shuffleId); + } else { + int uniffleShuffleId = + shuffleIdMappingManager.createUniffleShuffleId(shuffleId, appShuffleIdentifier); + int requiredShuffleServerNumber = + RssSparkShuffleUtils.getRequiredShuffleServerNumber(sparkConf); + int estimateTaskConcurrency = RssSparkShuffleUtils.estimateTaskConcurrency(sparkConf); + Map> partitionToServers = + requestShuffleAssignment( + uniffleShuffleId, + shuffleIdToPartitionNum.get(shuffleId), + 1, + requiredShuffleServerNumber, + estimateTaskConcurrency, + rssStageResubmitManager.getServerIdBlackList()); + MutableShuffleHandleInfo shuffleHandleInfo = + new MutableShuffleHandleInfo( + uniffleShuffleId, partitionToServers, getRemoteStorageInfo()); + shuffleHandleInfoManager.register(uniffleShuffleId, shuffleHandleInfo); + newShuffleId = uniffleShuffleId; + } + } + } else { + newShuffleId = shuffleIdMappingManager.getUniffleShuffleIdForRead(shuffleId); + } + return newShuffleId; } protected synchronized Supplier getOrCreateShuffleManagerClientSupplier() { @@ -961,41 +946,6 @@ public void addFailuresShuffleServerInfos(String shuffleServerId) { rssStageResubmitManager.recordFailuresShuffleServer(shuffleServerId); } - /** - * Reassign the ShuffleServer list for ShuffleId - * - * @param shuffleId - */ - @Override - public boolean reassignOnStageResubmit( - int shuffleId, int stageAttemptId, int stageAttemptNumber) { - int requiredShuffleServerNumber = - RssSparkShuffleUtils.getRequiredShuffleServerNumber(sparkConf); - int estimateTaskConcurrency = RssSparkShuffleUtils.estimateTaskConcurrency(sparkConf); - Map> partitionToServers = - requestShuffleAssignment( - shuffleId, - getPartitionNum(shuffleId), - 1, - requiredShuffleServerNumber, - estimateTaskConcurrency, - rssStageResubmitManager.getServerIdBlackList(), - stageAttemptId, - stageAttemptNumber, - false); - MutableShuffleHandleInfo shuffleHandleInfo = - new MutableShuffleHandleInfo( - shuffleId, partitionToServers, getRemoteStorageInfo(), partitionSplitMode); - StageAttemptShuffleHandleInfo stageAttemptShuffleHandleInfo = - (StageAttemptShuffleHandleInfo) shuffleHandleInfoManager.get(shuffleId); - stageAttemptShuffleHandleInfo.replaceCurrentShuffleHandleInfo(shuffleHandleInfo); - LOG.info( - "The stage retry has been triggered successfully for the shuffleId: {}, attemptNumber: {}", - shuffleId, - stageAttemptNumber); - return true; - } - /** this is only valid on driver side that exposed to being invoked by grpc server */ @Override public MutableShuffleHandleInfo reassignOnBlockSendFailure( @@ -1005,14 +955,7 @@ public MutableShuffleHandleInfo reassignOnBlockSendFailure( Map> partitionToFailureServers, boolean partitionSplit) { long startTime = System.currentTimeMillis(); - ShuffleHandleInfo handleInfo = shuffleHandleInfoManager.get(shuffleId); - MutableShuffleHandleInfo internalHandle = null; - if (handleInfo instanceof MutableShuffleHandleInfo) { - internalHandle = (MutableShuffleHandleInfo) handleInfo; - } else if (handleInfo instanceof StageAttemptShuffleHandleInfo) { - internalHandle = - (MutableShuffleHandleInfo) ((StageAttemptShuffleHandleInfo) handleInfo).getCurrent(); - } + MutableShuffleHandleInfo internalHandle = shuffleHandleInfoManager.get(shuffleId); if (internalHandle == null) { throw new RssException( "An unexpected error occurred: internalHandle is null, which should not happen"); @@ -1283,10 +1226,7 @@ private Set reassignServerForTask( .collect(Collectors.toSet()); replacementsRef.set(replacements); return createShuffleAssignmentsInfo(replacements, partitionIds); - }, - stageId, - stageAttemptNumber, - reassign); + }); return replacementsRef.get(); } @@ -1297,10 +1237,7 @@ private Map> requestShuffleAssignment( int assignmentShuffleServerNumber, int estimateTaskConcurrency, Set faultyServerIds, - Function reassignmentHandler, - int stageId, - int stageAttemptNumber, - boolean reassign) { + Function reassignmentHandler) { Set assignmentTags = RssSparkShuffleUtils.getAssignmentTags(sparkConf); ClientUtils.validateClientType(clientType); assignmentTags.add(clientType); @@ -1318,9 +1255,6 @@ private Map> requestShuffleAssignment( assignmentShuffleServerNumber, estimateTaskConcurrency, faultyServerIds, - stageId, - stageAttemptNumber, - reassign, retryInterval, retryTimes); LOG.info("Finished reassign"); @@ -1341,10 +1275,7 @@ protected Map> requestShuffleAssignment( int partitionNumPerRange, int assignmentShuffleServerNumber, int estimateTaskConcurrency, - Set faultyServerIds, - int stageId, - int stageAttemptNumber, - boolean reassign) { + Set faultyServerIds) { Set assignmentTags = RssSparkShuffleUtils.getAssignmentTags(sparkConf); ClientUtils.validateClientType(clientType); assignmentTags.add(clientType); @@ -1367,17 +1298,10 @@ protected Map> requestShuffleAssignment( assignmentShuffleServerNumber, estimateTaskConcurrency, faultyServerIds, - stageId, - stageAttemptNumber, - reassign, 0, 0); registerShuffleServers( - appId, - shuffleId, - response.getServerToPartitionRanges(), - getRemoteStorageInfo(), - stageAttemptNumber); + appId, shuffleId, response.getServerToPartitionRanges(), getRemoteStorageInfo()); return response.getPartitionToServers(); }, retryInterval, @@ -1387,32 +1311,11 @@ protected Map> requestShuffleAssignment( } } - protected Map> requestShuffleAssignment( - int shuffleId, - int partitionNum, - int partitionNumPerRange, - int assignmentShuffleServerNumber, - int estimateTaskConcurrency, - Set faultyServerIds, - int stageAttemptNumber) { - return requestShuffleAssignment( - shuffleId, - partitionNum, - partitionNumPerRange, - assignmentShuffleServerNumber, - estimateTaskConcurrency, - faultyServerIds, - -1, - stageAttemptNumber, - false); - } - protected void registerShuffleServers( String appId, int shuffleId, Map> serverToPartitionRanges, - RemoteStorageInfo remoteStorage, - int stageAttemptNumber) { + RemoteStorageInfo remoteStorage) { if (serverToPartitionRanges == null || serverToPartitionRanges.isEmpty()) { return; } @@ -1430,7 +1333,6 @@ protected void registerShuffleServers( remoteStorage, ShuffleDataDistributionType.NORMAL, maxConcurrencyPerPartitionToWrite, - stageAttemptNumber, null, sparkConfMap); }); @@ -1438,39 +1340,6 @@ protected void registerShuffleServers( "Finish register shuffleId {} with {} ms", shuffleId, (System.currentTimeMillis() - start)); } - @VisibleForTesting - protected void registerShuffleServers( - String appId, - int shuffleId, - Map> serverToPartitionRanges, - RemoteStorageInfo remoteStorage) { - if (serverToPartitionRanges == null || serverToPartitionRanges.isEmpty()) { - return; - } - LOG.info("Start to register shuffleId[{}]", shuffleId); - long start = System.currentTimeMillis(); - Map sparkConfMap = sparkConfToMap(getSparkConf()); - Set>> entries = - serverToPartitionRanges.entrySet(); - entries.stream() - .forEach( - entry -> { - shuffleWriteClient.registerShuffle( - entry.getKey(), - appId, - shuffleId, - entry.getValue(), - remoteStorage, - dataDistributionType, - maxConcurrencyPerPartitionToWrite, - sparkConfMap); - }); - LOG.info( - "Finish register shuffleId[{}] with {} ms", - shuffleId, - (System.currentTimeMillis() - start)); - } - protected RemoteStorageInfo getRemoteStorageInfo() { String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key()); RemoteStorageInfo defaultRemoteStorage = @@ -1483,14 +1352,6 @@ public boolean isRssStageRetryEnabled() { return rssStageRetryEnabled; } - public boolean isRssStageRetryForWriteFailureEnabled() { - return rssStageRetryForWriteFailureEnabled; - } - - public boolean isRssStageRetryForFetchFailureEnabled() { - return rssStageRetryForFetchFailureEnabled; - } - @VisibleForTesting public SparkConf getSparkConf() { return sparkConf; diff --git a/client-spark/common/src/main/java/org/apache/uniffle/shuffle/manager/RssShuffleManagerInterface.java b/client-spark/common/src/main/java/org/apache/uniffle/shuffle/manager/RssShuffleManagerInterface.java index 82967cc4a3..47e0523390 100644 --- a/client-spark/common/src/main/java/org/apache/uniffle/shuffle/manager/RssShuffleManagerInterface.java +++ b/client-spark/common/src/main/java/org/apache/uniffle/shuffle/manager/RssShuffleManagerInterface.java @@ -81,8 +81,6 @@ public interface RssShuffleManagerInterface { */ void addFailuresShuffleServerInfos(String shuffleServerId); - boolean reassignOnStageResubmit(int shuffleId, int stageAttemptId, int stageAttemptNumber); - MutableShuffleHandleInfo reassignOnBlockSendFailure( int stageId, int stageAttemptNumber, @@ -96,4 +94,6 @@ MutableShuffleHandleInfo reassignOnBlockSendFailure( * @return */ ShuffleWriteClient getShuffleWriteClient(); + + int getUniffleShuffleId(int shuffleId, String appShuffleIdentifier, boolean isWriter); } diff --git a/client-spark/common/src/main/java/org/apache/uniffle/shuffle/manager/ShuffleManagerGrpcService.java b/client-spark/common/src/main/java/org/apache/uniffle/shuffle/manager/ShuffleManagerGrpcService.java index 213f1a774f..23a17816a8 100644 --- a/client-spark/common/src/main/java/org/apache/uniffle/shuffle/manager/ShuffleManagerGrpcService.java +++ b/client-spark/common/src/main/java/org/apache/uniffle/shuffle/manager/ShuffleManagerGrpcService.java @@ -17,13 +17,9 @@ package org.apache.uniffle.shuffle.manager; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -37,13 +33,11 @@ import org.apache.spark.shuffle.events.TaskShuffleReadInfoEvent; import org.apache.spark.shuffle.events.TaskShuffleWriteInfoEvent; import org.apache.spark.shuffle.handle.MutableShuffleHandleInfo; -import org.apache.spark.shuffle.handle.StageAttemptShuffleHandleInfo; import org.roaringbitmap.longlong.Roaring64NavigableMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.uniffle.common.ReceivingFailureServer; -import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.exception.RssException; import org.apache.uniffle.common.util.JavaUtils; import org.apache.uniffle.common.util.RssUtils; @@ -54,117 +48,21 @@ public class ShuffleManagerGrpcService extends ShuffleManagerImplBase { private static final Logger LOG = LoggerFactory.getLogger(ShuffleManagerGrpcService.class); private final Map shuffleStatus = JavaUtils.newConcurrentMap(); - // The shuffleId mapping records the number of ShuffleServer write failures - private final Map shuffleWriteStatus = - JavaUtils.newConcurrentMap(); private final RssShuffleManagerInterface shuffleManager; public ShuffleManagerGrpcService(RssShuffleManagerInterface shuffleManager) { this.shuffleManager = shuffleManager; } - @Override - public void reportShuffleWriteFailure( - RssProtos.ReportShuffleWriteFailureRequest request, - StreamObserver responseObserver) { - String appId = request.getAppId(); - int shuffleId = request.getShuffleId(); - int stageAttemptId = request.getStageAttemptId(); - int stageAttemptNumber = request.getStageAttemptNumber(); - List shuffleServerIdsList = request.getShuffleServerIdsList(); - RssProtos.StatusCode code; - boolean reSubmitWholeStage; - String msg; - if (!appId.equals(shuffleManager.getAppId())) { - msg = - String.format( - "got a wrong shuffle write failure report from appId: %s, expected appId: %s", - appId, shuffleManager.getAppId()); - LOG.warn(msg); - code = RssProtos.StatusCode.INVALID_REQUEST; - reSubmitWholeStage = false; - } else { - Map initServerFailures = JavaUtils.newConcurrentMap(); - List shuffleServerInfos = - ShuffleServerInfo.fromProto(shuffleServerIdsList); - shuffleServerInfos.forEach( - shuffleServerInfo -> - initServerFailures.computeIfAbsent( - shuffleServerInfo.getId(), key -> new AtomicInteger(0))); - ShuffleServerWriterFailureRecord shuffleServerWriterFailureRecord = - shuffleWriteStatus.computeIfAbsent( - shuffleId, - key -> new ShuffleServerWriterFailureRecord(stageAttemptNumber, initServerFailures)); - boolean resetflag = - shuffleServerWriterFailureRecord.resetStageAttemptIfNecessary(stageAttemptNumber); - if (resetflag) { - msg = - String.format( - "got an old stage(%d_%d) shuffle write failure report, which should be impossible.", - stageAttemptId, stageAttemptNumber); - LOG.warn(msg); - code = RssProtos.StatusCode.INVALID_REQUEST; - reSubmitWholeStage = false; - } else { - synchronized (shuffleServerWriterFailureRecord) { - code = RssProtos.StatusCode.SUCCESS; - // update the stage shuffleServer write failed count - boolean isFetchFailed = - shuffleServerWriterFailureRecord.incWriteFailureForShuffleServer( - stageAttemptNumber, shuffleServerInfos, shuffleManager); - if (isFetchFailed) { - reSubmitWholeStage = true; - msg = - String.format( - "Report shuffle write failure as maximum number(%d) of shuffle write is occurred.", - shuffleManager.getMaxFetchFailures()); - if (!shuffleServerWriterFailureRecord.isClearedMapTrackerBlock()) { - try { - // Clear the metadata of the completed task, otherwise some of the stage's data will - // be lost. - shuffleManager.unregisterAllMapOutput(shuffleId); - // Deregister the shuffleId corresponding to the Shuffle Server. - shuffleManager.getShuffleWriteClient().unregisterShuffle(appId, shuffleId); - shuffleServerWriterFailureRecord.setClearedMapTrackerBlock(true); - LOG.info( - "Clear shuffle result in shuffleId:{}, stageId:{}, stageAttemptNumber:{} in the write failure phase.", - shuffleId, - stageAttemptId, - stageAttemptNumber); - } catch (SparkException e) { - LOG.error( - "Clear MapoutTracker Meta failed in shuffleId:{}, stageAttemptId:{}, stageAttemptNumber:{} in the write failure phase.", - shuffleId, - stageAttemptId, - stageAttemptNumber); - throw new RssException("Clear MapoutTracker Meta failed!", e); - } - } - } else { - reSubmitWholeStage = false; - msg = "The maximum number of failures was not reached."; - } - } - } - } - - RssProtos.ReportShuffleWriteFailureResponse reply = - RssProtos.ReportShuffleWriteFailureResponse.newBuilder() - .setStatus(code) - .setReSubmitWholeStage(reSubmitWholeStage) - .setMsg(msg) - .build(); - responseObserver.onNext(reply); - responseObserver.onCompleted(); - } - @Override public void reportShuffleFetchFailure( RssProtos.ReportShuffleFetchFailureRequest request, StreamObserver responseObserver) { String appId = request.getAppId(); int shuffleId = request.getShuffleId(); - int stageAttempt = request.getStageAttemptId(); + int uniffleShuffleId = request.getUniffleShuffleId(); + int stageAttemptId = request.getStageAttemptId(); + int stageAttemptNumber = request.getStageAttemptNumber(); int partitionId = request.getPartitionId(); RssProtos.StatusCode code; boolean reSubmitWholeStage; @@ -183,22 +81,23 @@ public void reportShuffleFetchFailure( request.getShuffleId(), key -> { int partitionNum = shuffleManager.getPartitionNum(key); - return new RssShuffleStatus(partitionNum, stageAttempt); + return new RssShuffleStatus(partitionNum, stageAttemptNumber); }); - int c = status.resetStageAttemptIfNecessary(stageAttempt); + int c = status.resetStageAttemptIfNecessary(stageAttemptNumber); if (c < 0) { msg = String.format( "got an old stage(%d vs %d) shuffle fetch failure report, which should be impossible.", - status.getStageAttempt(), stageAttempt); + status.getStageAttempt(), stageAttemptNumber); LOG.warn(msg); code = RssProtos.StatusCode.INVALID_REQUEST; reSubmitWholeStage = false; } else { // update the stage partition fetch failure count synchronized (status) { code = RssProtos.StatusCode.SUCCESS; - status.incPartitionFetchFailure(stageAttempt, partitionId); - if (status.currentPartitionIsFetchFailed(stageAttempt, partitionId, shuffleManager)) { + status.incPartitionFetchFailure(stageAttemptNumber, partitionId); + if (status.currentPartitionIsFetchFailed( + stageAttemptNumber, partitionId, shuffleManager)) { reSubmitWholeStage = true; if (!status.hasClearedMapTrackerBlock()) { try { @@ -207,14 +106,18 @@ public void reportShuffleFetchFailure( shuffleManager.unregisterAllMapOutput(shuffleId); status.clearedMapTrackerBlock(); LOG.info( - "Clear shuffle result in shuffleId:{}, stageId:{} in the write failure phase.", + "Clear shuffle result in shuffleId:{}, uniffleShuffleId:{}, stageId:{}, stageAttemptNumber:{} in the write failure phase.", shuffleId, - stageAttempt); + uniffleShuffleId, + stageAttemptId, + stageAttemptNumber); } catch (SparkException e) { LOG.error( - "Clear MapoutTracker Meta failed in shuffleId:{}, stageAttemptId:{} in the write failure phase.", + "Clear MapoutTracker Meta failed in shuffleId:{}, uniffleShuffleId:{}, stageId:{}, stageAttemptNumber:{} in the write failure phase.", shuffleId, - stageAttempt); + uniffleShuffleId, + stageAttemptId, + stageAttemptNumber); throw new RssException("Clear MapoutTracker Meta failed!", e); } } @@ -240,72 +143,6 @@ public void reportShuffleFetchFailure( responseObserver.onCompleted(); } - @Override - public void getPartitionToShufflerServerWithStageRetry( - RssProtos.PartitionToShuffleServerRequest request, - StreamObserver responseObserver) { - RssProtos.ReassignOnStageRetryResponse reply; - RssProtos.StatusCode code; - int stageAttemptId = request.getStageAttemptId(); - int stageAttemptNumber = request.getStageAttemptNumber(); - int shuffleId = request.getShuffleId(); - boolean isWritePhase = request.getIsWritePhase(); - StageAttemptShuffleHandleInfo shuffleHandle; - if (isWritePhase) { - ShuffleServerWriterFailureRecord shuffleServerWriterFailureRecord = - shuffleWriteStatus.get(shuffleId); - if (shuffleServerWriterFailureRecord != null) { - synchronized (shuffleServerWriterFailureRecord) { - if (shuffleServerWriterFailureRecord.isNeedReassignForLastStageNumber( - stageAttemptNumber)) { - shuffleManager.reassignOnStageResubmit(shuffleId, stageAttemptId, stageAttemptNumber); - shuffleServerWriterFailureRecord.setShuffleServerAssignmented(true); - } - } - } - } - shuffleHandle = - (StageAttemptShuffleHandleInfo) shuffleManager.getShuffleHandleInfoByShuffleId(shuffleId); - if (shuffleHandle != null) { - code = RssProtos.StatusCode.SUCCESS; - reply = - RssProtos.ReassignOnStageRetryResponse.newBuilder() - .setStatus(code) - .setShuffleHandleInfo(StageAttemptShuffleHandleInfo.toProto(shuffleHandle)) - .build(); - } else { - code = RssProtos.StatusCode.INVALID_REQUEST; - reply = RssProtos.ReassignOnStageRetryResponse.newBuilder().setStatus(code).build(); - } - - responseObserver.onNext(reply); - responseObserver.onCompleted(); - } - - @Override - public void getPartitionToShufflerServerWithBlockRetry( - RssProtos.PartitionToShuffleServerRequest request, - StreamObserver responseObserver) { - RssProtos.ReassignOnBlockSendFailureResponse reply; - RssProtos.StatusCode code; - int shuffleId = request.getShuffleId(); - MutableShuffleHandleInfo shuffleHandle = - (MutableShuffleHandleInfo) shuffleManager.getShuffleHandleInfoByShuffleId(shuffleId); - if (shuffleHandle != null) { - code = RssProtos.StatusCode.SUCCESS; - reply = - RssProtos.ReassignOnBlockSendFailureResponse.newBuilder() - .setStatus(code) - .setHandle(MutableShuffleHandleInfo.toProto(shuffleHandle)) - .build(); - } else { - code = RssProtos.StatusCode.INVALID_REQUEST; - reply = RssProtos.ReassignOnBlockSendFailureResponse.newBuilder().setStatus(code).build(); - } - responseObserver.onNext(reply); - responseObserver.onCompleted(); - } - @Override public void reassignOnBlockSendFailure( org.apache.uniffle.proto.RssProtos.RssReassignOnBlockSendFailureRequest request, @@ -361,143 +198,6 @@ public void unregisterShuffle(int shuffleId) { shuffleStatus.remove(shuffleId); } - private static class ShuffleServerWriterFailureRecord { - private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - private final ReentrantReadWriteLock.ReadLock readLock = lock.readLock(); - private final ReentrantReadWriteLock.WriteLock writeLock = lock.writeLock(); - private final Map shuffleServerWriteFailureRecordCount; - private Integer stageAttemptNumber; - // Whether the ShuffleServer has been reassigned for the current number of attempts. - private boolean isShuffleServerAssignmented; - // Whether a retry is required for the current number of attempts. - private boolean isStageNeedRetry; - // Whether the Shuffle result has been cleared for the current number of attempts. - private boolean isClearedMapTrackerBlock; - - private ShuffleServerWriterFailureRecord( - Integer stageAttemptNumber, Map initServerFailures) { - this.stageAttemptNumber = stageAttemptNumber; - this.shuffleServerWriteFailureRecordCount = initServerFailures; - this.isStageNeedRetry = false; - this.isShuffleServerAssignmented = false; - this.isClearedMapTrackerBlock = false; - } - - private T withReadLock(Supplier fn) { - readLock.lock(); - try { - return fn.get(); - } finally { - readLock.unlock(); - } - } - - private T withWriteLock(Supplier fn) { - writeLock.lock(); - try { - return fn.get(); - } finally { - writeLock.unlock(); - } - } - - public boolean resetStageAttemptIfNecessary(int stageAttemptNumber) { - return withWriteLock( - () -> { - if (this.stageAttemptNumber < stageAttemptNumber) { - // a new stage attempt is issued. Record the shuffleServer status of the Map should be - // clear and reset. - this.shuffleServerWriteFailureRecordCount.clear(); - this.stageAttemptNumber = stageAttemptNumber; - this.isStageNeedRetry = false; - this.isShuffleServerAssignmented = false; - this.isClearedMapTrackerBlock = false; - return false; - } - return this.stageAttemptNumber > stageAttemptNumber; - }); - } - - public boolean incWriteFailureForShuffleServer( - int stageAttemptNumber, - List shuffleServerInfos, - RssShuffleManagerInterface shuffleManager) { - return withWriteLock( - () -> { - if (this.stageAttemptNumber != stageAttemptNumber) { - // If it is not the latest StageAttemptNumber, skip it. - return false; - } else { - shuffleServerInfos.forEach( - shuffleServerInfo -> - shuffleServerWriteFailureRecordCount - .computeIfAbsent(shuffleServerInfo.getId(), key -> new AtomicInteger()) - .incrementAndGet()); - List> serverFailuresList = - new ArrayList(shuffleServerWriteFailureRecordCount.entrySet()); - if (serverFailuresList.isEmpty()) { - return false; - } else { - if (isStageNeedRetry) { - // You've decided to throw fetchfailed, you just throw it. - return true; - } else { - int failureCnt = 0; - // They need to be sorted in reverse order, and then the Shuffleserver that - // exceeds - // the maximum number of failures is selected. - Collections.sort( - serverFailuresList, Comparator.comparingInt(o -> -o.getValue().get())); - for (int i = 0; i < serverFailuresList.size(); i++) { - Map.Entry serverFailure = serverFailuresList.get(i); - if (serverFailure.getValue().get() > shuffleManager.getMaxFetchFailures()) { - shuffleManager.addFailuresShuffleServerInfos(serverFailure.getKey()); - failureCnt++; - } else { - break; - } - } - if (failureCnt > 0) { - this.isStageNeedRetry = true; - return true; - } else { - return false; - } - } - } - } - }); - } - - public boolean isNeedReassignForLastStageNumber(int lastStageAttemptNumber) { - return withReadLock( - () -> - isStageNeedRetry - && !isShuffleServerAssignmented - && stageAttemptNumber == lastStageAttemptNumber - 1); - } - - public void setShuffleServerAssignmented(boolean isAssignmented) { - withWriteLock( - () -> { - this.isShuffleServerAssignmented = isAssignmented; - return null; - }); - } - - public void setClearedMapTrackerBlock(boolean isCleared) { - withWriteLock( - () -> { - this.isClearedMapTrackerBlock = isCleared; - return null; - }); - } - - public boolean isClearedMapTrackerBlock() { - return withReadLock(() -> isClearedMapTrackerBlock); - } - } - private static class RssShuffleStatus { private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); private final ReentrantReadWriteLock.ReadLock readLock = lock.readLock(); @@ -773,4 +473,45 @@ public void reportShuffleReadMetric( responseObserver.onNext(reply); responseObserver.onCompleted(); } + + @Override + public void getPartitionToShufflerServer( + RssProtos.PartitionToShuffleServerRequest request, + StreamObserver responseObserver) { + RssProtos.ReassignOnBlockSendFailureResponse reply; + RssProtos.StatusCode code; + int shuffleId = request.getShuffleId(); + MutableShuffleHandleInfo shuffleHandle = + (MutableShuffleHandleInfo) shuffleManager.getShuffleHandleInfoByShuffleId(shuffleId); + if (shuffleHandle != null) { + code = RssProtos.StatusCode.SUCCESS; + reply = + RssProtos.ReassignOnBlockSendFailureResponse.newBuilder() + .setStatus(code) + .setHandle(MutableShuffleHandleInfo.toProto(shuffleHandle)) + .build(); + } else { + code = RssProtos.StatusCode.INVALID_REQUEST; + reply = RssProtos.ReassignOnBlockSendFailureResponse.newBuilder().setStatus(code).build(); + } + responseObserver.onNext(reply); + responseObserver.onCompleted(); + } + + @Override + public void getUniffleShuffleId( + RssProtos.AppUniffleShuffleIdRequest request, + StreamObserver responseObserver) { + int shuffleId = request.getShuffleId(); + String appShuffleIdentifier = request.getAppShuffleIdentifier(); + boolean isWriter = request.getIsWriter(); + int uniffleShuffleId = + shuffleManager.getUniffleShuffleId(shuffleId, appShuffleIdentifier, isWriter); + RssProtos.AppUniffleShuffleIdResponse reply = + RssProtos.AppUniffleShuffleIdResponse.newBuilder() + .setGeneratorShuffleId(uniffleShuffleId) + .build(); + responseObserver.onNext(reply); + responseObserver.onCompleted(); + } } diff --git a/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/DataPusherTest.java b/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/DataPusherTest.java index 8ac07664f9..e4ad117734 100644 --- a/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/DataPusherTest.java +++ b/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/DataPusherTest.java @@ -71,15 +71,6 @@ public SendShuffleDataResult sendShuffleData( String appId, List shuffleBlockInfoList, Supplier needCancelRequest) { - return sendShuffleData(appId, 0, shuffleBlockInfoList, needCancelRequest); - } - - @Override - public SendShuffleDataResult sendShuffleData( - String appId, - int stageAttemptNumber, - List shuffleBlockInfoList, - Supplier needCancelRequest) { return fakedShuffleDataResult; } diff --git a/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/WriteBufferManagerTest.java b/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/WriteBufferManagerTest.java index 501b57e444..2ea0347f65 100644 --- a/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/WriteBufferManagerTest.java +++ b/client-spark/common/src/test/java/org/apache/spark/shuffle/writer/WriteBufferManagerTest.java @@ -333,8 +333,7 @@ public void spillByOthersTest() { mockTaskMemoryManager, new ShuffleWriteMetrics(), RssSparkConfig.toRssConf(conf), - null, - 0); + null); WriteBufferManager spyManager = spy(wbm); doReturn(512L).when(spyManager).acquireMemory(anyLong()); @@ -367,8 +366,7 @@ public void spillByOwnTest() { mockTaskMemoryManager, new ShuffleWriteMetrics(), RssSparkConfig.toRssConf(conf), - null, - 0); + null); Function, List>> spillFunc = blocks -> { @@ -478,8 +476,7 @@ public void spillPartial() { mockTaskMemoryManager, new ShuffleWriteMetrics(), RssSparkConfig.toRssConf(conf), - null, - 0); + null); Function, List>> spillFunc = blocks -> { @@ -574,8 +571,7 @@ public void spillByOwnWithSparkTaskMemoryManagerTest() { fakedTaskMemoryManager, new ShuffleWriteMetrics(), RssSparkConfig.toRssConf(conf), - null, - 0); + null); List blockList = new ArrayList<>(); diff --git a/client-spark/common/src/test/java/org/apache/uniffle/shuffle/manager/DummyRssShuffleManager.java b/client-spark/common/src/test/java/org/apache/uniffle/shuffle/manager/DummyRssShuffleManager.java index d82581915e..736aab55b2 100644 --- a/client-spark/common/src/test/java/org/apache/uniffle/shuffle/manager/DummyRssShuffleManager.java +++ b/client-spark/common/src/test/java/org/apache/uniffle/shuffle/manager/DummyRssShuffleManager.java @@ -70,12 +70,6 @@ public int getMaxFetchFailures() { @Override public void addFailuresShuffleServerInfos(String shuffleServerId) {} - @Override - public boolean reassignOnStageResubmit( - int shuffleId, int stageAttemptId, int stageAttemptNumber) { - return false; - } - @Override public MutableShuffleHandleInfo reassignOnBlockSendFailure( int stageId, @@ -90,4 +84,9 @@ public MutableShuffleHandleInfo reassignOnBlockSendFailure( public ShuffleWriteClient getShuffleWriteClient() { return null; } + + @Override + public int getUniffleShuffleId(int shuffleId, String appShuffleIdentifier, boolean isWriter) { + return 0; + } } diff --git a/client-spark/common/src/test/java/org/apache/uniffle/shuffle/manager/ShuffleManagerGrpcServiceTest.java b/client-spark/common/src/test/java/org/apache/uniffle/shuffle/manager/ShuffleManagerGrpcServiceTest.java index ac3fbda7e3..e1e687b538 100644 --- a/client-spark/common/src/test/java/org/apache/uniffle/shuffle/manager/ShuffleManagerGrpcServiceTest.java +++ b/client-spark/common/src/test/java/org/apache/uniffle/shuffle/manager/ShuffleManagerGrpcServiceTest.java @@ -24,8 +24,6 @@ import org.apache.uniffle.proto.RssProtos.ReportShuffleFetchFailureRequest; import org.apache.uniffle.proto.RssProtos.ReportShuffleFetchFailureResponse; -import org.apache.uniffle.proto.RssProtos.ReportShuffleWriteFailureRequest; -import org.apache.uniffle.proto.RssProtos.ReportShuffleWriteFailureResponse; import org.apache.uniffle.proto.RssProtos.StatusCode; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -81,7 +79,9 @@ public void testShuffleManagerGrpcService() { ReportShuffleFetchFailureRequest.newBuilder() .setAppId(appId) .setShuffleId(shuffleId) + .setUniffleShuffleId(shuffleId) .setStageAttemptId(1) + .setStageAttemptNumber(1) .setPartitionId(1) .buildPartial(); @@ -104,22 +104,11 @@ public void testShuffleManagerGrpcService() { ReportShuffleFetchFailureRequest.newBuilder() .mergeFrom(req) .setAppId(appId) - .setStageAttemptId(0) + .setStageAttemptId(1) + .setStageAttemptNumber(0) .build(); service.reportShuffleFetchFailure(req, appIdResponseObserver); assertEquals(StatusCode.INVALID_REQUEST, appIdResponseObserver.value.getStatus()); assertTrue(appIdResponseObserver.value.getMsg().contains("old stage")); - - // reportShuffleWriteFailure with an empty list of shuffleServerIds - MockedStreamObserver - reportShuffleWriteFailureResponseObserver = new MockedStreamObserver<>(); - ReportShuffleWriteFailureRequest reportShuffleWriteFailureRequest = - ReportShuffleWriteFailureRequest.newBuilder() - .setAppId(appId) - .setShuffleId(shuffleId) - .buildPartial(); - service.reportShuffleWriteFailure( - reportShuffleWriteFailureRequest, reportShuffleWriteFailureResponseObserver); - assertEquals(StatusCode.SUCCESS, reportShuffleWriteFailureResponseObserver.value.getStatus()); } } diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java index 31521a4b21..8caa1341e9 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java @@ -36,10 +36,9 @@ import org.apache.spark.TaskContext; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.executor.ShuffleWriteMetrics; -import org.apache.spark.shuffle.handle.MutableShuffleHandleInfo; +import org.apache.spark.rdd.DeterministicLevel; import org.apache.spark.shuffle.handle.ShuffleHandleInfo; import org.apache.spark.shuffle.handle.SimpleShuffleHandleInfo; -import org.apache.spark.shuffle.handle.StageAttemptShuffleHandleInfo; import org.apache.spark.shuffle.reader.RssShuffleReader; import org.apache.spark.shuffle.writer.RssShuffleWriter; import org.apache.spark.storage.BlockId; @@ -113,15 +112,21 @@ public ShuffleHandle registerShuffle( LOG.info("Generate application id used in rss: " + appId); } + // If stage retry is enabled, the Deterministic status of the ShuffleId needs to be recorded. + if (rssStageRetryEnabled) { + shuffleIdMappingManager.recordShuffleIdDeterminate( + shuffleId, + dependency.rdd().getOutputDeterministicLevel() != DeterministicLevel.INDETERMINATE()); + } + if (dependency.partitioner().numPartitions() == 0) { shuffleIdToPartitionNum.putIfAbsent(shuffleId, 0); shuffleIdToNumMapTasks.computeIfAbsent( shuffleId, key -> dependency.rdd().partitions().length); LOG.info( - "RegisterShuffle with ShuffleId[" - + shuffleId - + "], partitionNum is 0, " - + "return the empty RssShuffleHandle directly"); + "RegisterShuffle with ShuffleId[{}], partitionNum is 0, " + + "return the empty RssShuffleHandle directly", + shuffleId); Broadcast hdlInfoBd = RssSparkShuffleUtils.broadcastShuffleHdlInfo( RssSparkShuffleUtils.getActiveSparkContext(), @@ -132,12 +137,6 @@ public ShuffleHandle registerShuffle( shuffleId, appId, dependency.rdd().getNumPartitions(), dependency, hdlInfoBd); } - String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key()); - RemoteStorageInfo defaultRemoteStorage = getDefaultRemoteStorageInfo(sparkConf); - RemoteStorageInfo remoteStorage = - ClientUtils.fetchRemoteStorage( - appId, defaultRemoteStorage, dynamicConfEnabled, storageType, shuffleWriteClient); - // get all register info according to coordinator's response Set assignmentTags = RssSparkShuffleUtils.getAssignmentTags(sparkConf); ClientUtils.validateClientType(clientType); @@ -147,45 +146,44 @@ public ShuffleHandle registerShuffle( RssSparkShuffleUtils.getRequiredShuffleServerNumber(sparkConf); int estimateTaskConcurrency = RssSparkShuffleUtils.estimateTaskConcurrency(sparkConf); - Map> partitionToServers = - requestShuffleAssignment( - shuffleId, - dependency.partitioner().numPartitions(), - 1, - requiredShuffleServerNumber, - estimateTaskConcurrency, - rssStageResubmitManager.getServerIdBlackList(), - 0); - + Map> partitionToServers = null; + if (!rssStageRetryEnabled) { + // The ShuffleServer will be allocated here only when the Spark phase retry is not enabled; + // otherwise, the ShuffleServer will be allocated and registered during the Shuffle write + // phase. + partitionToServers = + requestShuffleAssignment( + shuffleId, + dependency.partitioner().numPartitions(), + 1, + requiredShuffleServerNumber, + estimateTaskConcurrency, + rssStageResubmitManager.getServerIdBlackList()); + } startHeartbeat(); shuffleIdToPartitionNum.computeIfAbsent( shuffleId, key -> dependency.partitioner().numPartitions()); shuffleIdToNumMapTasks.computeIfAbsent(shuffleId, key -> dependency.rdd().partitions().length); - if (shuffleManagerRpcServiceEnabled && rssStageRetryForWriteFailureEnabled) { - ShuffleHandleInfo handleInfo = - new MutableShuffleHandleInfo(shuffleId, partitionToServers, remoteStorage); - StageAttemptShuffleHandleInfo stageAttemptShuffleHandleInfo = - new StageAttemptShuffleHandleInfo(shuffleId, remoteStorage, handleInfo); - shuffleHandleInfoManager.register(shuffleId, stageAttemptShuffleHandleInfo); - } else if (shuffleManagerRpcServiceEnabled && partitionReassignEnabled) { - ShuffleHandleInfo shuffleHandleInfo = - new MutableShuffleHandleInfo(shuffleId, partitionToServers, remoteStorage); - shuffleHandleInfoManager.register(shuffleId, shuffleHandleInfo); - } + String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key()); + RemoteStorageInfo defaultRemoteStorage = getDefaultRemoteStorageInfo(sparkConf); + RemoteStorageInfo remoteStorage = + ClientUtils.fetchRemoteStorage( + appId, defaultRemoteStorage, dynamicConfEnabled, storageType, shuffleWriteClient); + Broadcast hdlInfoBd = RssSparkShuffleUtils.broadcastShuffleHdlInfo( RssSparkShuffleUtils.getActiveSparkContext(), shuffleId, partitionToServers, remoteStorage); - LOG.info( - "RegisterShuffle with ShuffleId[" - + shuffleId - + "], partitionNum[" - + partitionToServers.size() - + "], server:{}", - partitionToServers); + if (partitionToServers != null) { + LOG.info( + "RegisterShuffle with ShuffleId[{}], partitionNum[{}], server:{}", + shuffleId, + partitionToServers.size(), + partitionToServers); + } return new RssShuffleHandle(shuffleId, appId, numMaps, dependency, hdlInfoBd); } @@ -200,11 +198,19 @@ public ShuffleWriter getWriter( dataPusher.setRssAppId(appId); int shuffleId = rssHandle.getShuffleId(); + int uniffleShuffleId; + if (rssStageRetryEnabled) { + uniffleShuffleId = getUniffleShuffleId(shuffleId, context, true); + } else { + uniffleShuffleId = shuffleId; + } + String taskId = "" + context.taskAttemptId() + "_" + context.attemptNumber(); ShuffleWriteMetrics writeMetrics = context.taskMetrics().shuffleWriteMetrics(); return new RssShuffleWriter<>( rssHandle.getAppId(), shuffleId, + uniffleShuffleId, taskId, getTaskAttemptIdForBlockId(context.partitionId(), context.attemptNumber()), writeMetrics, @@ -234,34 +240,20 @@ public ShuffleReader getReader( Roaring64NavigableMap taskIdBitmap = getExpectedTasks(shuffleId, startPartition, endPartition); LOG.info( - "Get taskId cost " - + (System.currentTimeMillis() - start) - + " ms, and request expected blockIds from " - + taskIdBitmap.getLongCardinality() - + " tasks for shuffleId[" - + shuffleId - + "], partitionId[" - + startPartition - + "]"); + "Get taskId cost {} ms, and request expected blockIds from {} tasks for shuffleId[{}], partitionId[{}]", + (System.currentTimeMillis() - start), + taskIdBitmap.getLongCardinality(), + shuffleId, + startPartition); start = System.currentTimeMillis(); - ShuffleHandleInfo shuffleHandleInfo; - if (shuffleManagerRpcServiceEnabled && rssStageRetryForWriteFailureEnabled) { - // In Stage Retry mode, Get the ShuffleServer list from the Driver based on the shuffleId. - shuffleHandleInfo = - getRemoteShuffleHandleInfoWithStageRetry( - context.stageId(), context.stageAttemptNumber(), shuffleId, false); - } else if (shuffleManagerRpcServiceEnabled && partitionReassignEnabled) { - // In Block Retry mode, Get the ShuffleServer list from the Driver based on the shuffleId - shuffleHandleInfo = - getRemoteShuffleHandleInfoWithBlockRetry( - context.stageId(), context.stageAttemptNumber(), shuffleId, false); + int uniffleShuffleId; + if (rssStageRetryEnabled) { + uniffleShuffleId = getUniffleShuffleId(shuffleId, context, false); } else { - shuffleHandleInfo = - new SimpleShuffleHandleInfo( - shuffleId, - rssShuffleHandle.getPartitionToServers(), - rssShuffleHandle.getRemoteStorage()); + uniffleShuffleId = shuffleId; } + ShuffleHandleInfo shuffleHandleInfo = + getShuffleHandleInfo(uniffleShuffleId, rssShuffleHandle); Map> partitionToServers = shuffleHandleInfo.getAllPartitionServersForReader(); Roaring64NavigableMap blockIdBitmap = @@ -270,18 +262,17 @@ public ShuffleReader getReader( Sets.newHashSet(partitionToServers.get(startPartition)), rssShuffleHandle.getAppId(), shuffleId, + uniffleShuffleId, startPartition, + context.stageId(), context.stageAttemptNumber()); LOG.info( - "Get shuffle blockId cost " - + (System.currentTimeMillis() - start) - + " ms, and get " - + blockIdBitmap.getLongCardinality() - + " blockIds for shuffleId[" - + shuffleId - + "], partitionId[" - + startPartition - + "]"); + "Get shuffle blockId cost {} ms, and get {} blockIds for shuffleId[{}], uniffleShuffleId[{}],partitionId[{}]", + (System.currentTimeMillis() - start), + blockIdBitmap.getLongCardinality(), + shuffleId, + uniffleShuffleId, + startPartition); final RemoteStorageInfo shuffleRemoteStorageInfo = rssShuffleHandle.getRemoteStorage(); LOG.info("Shuffle reader using remote storage {}", shuffleRemoteStorageInfo); @@ -290,6 +281,7 @@ public ShuffleReader getReader( RssSparkShuffleUtils.getRemoteStorageHadoopConf(sparkConf, shuffleRemoteStorageInfo); return new RssShuffleReader( + uniffleShuffleId, startPartition, endPartition, context, @@ -364,11 +356,13 @@ private Roaring64NavigableMap getShuffleResult( Set shuffleServerInfoSet, String appId, int shuffleId, + int uniffleShuffleId, int partitionId, - int stageAttemptId) { + int stageAttemptId, + int stageAttemptNumber) { try { return shuffleWriteClient.getShuffleResult( - clientType, shuffleServerInfoSet, appId, shuffleId, partitionId); + clientType, shuffleServerInfoSet, appId, uniffleShuffleId, partitionId); } catch (RssFetchFailedException e) { throw RssSparkShuffleUtils.reportRssFetchFailedException( managerClientSupplier, @@ -376,7 +370,9 @@ private Roaring64NavigableMap getShuffleResult( sparkConf, appId, shuffleId, + uniffleShuffleId, stageAttemptId, + stageAttemptNumber, Sets.newHashSet(partitionId)); } } @@ -385,7 +381,7 @@ private ShuffleServerInfo assignShuffleServer(int shuffleId, String faultyShuffl Set faultyServerIds = Sets.newHashSet(faultyShuffleServerId); faultyServerIds.addAll(rssStageResubmitManager.getServerIdBlackList()); Map> partitionToServers = - requestShuffleAssignment(shuffleId, 1, 1, 1, 1, faultyServerIds, 0); + requestShuffleAssignment(shuffleId, 1, 1, 1, 1, faultyServerIds); if (partitionToServers.get(0) != null && partitionToServers.get(0).size() == 1) { return partitionToServers.get(0).get(0); } diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/reader/RssShuffleReader.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/reader/RssShuffleReader.java index 4b4ec32c59..789e912899 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/reader/RssShuffleReader.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/reader/RssShuffleReader.java @@ -39,6 +39,7 @@ import org.apache.spark.executor.TempShuffleReadMetrics; import org.apache.spark.serializer.Serializer; import org.apache.spark.shuffle.RssShuffleHandle; +import org.apache.spark.shuffle.RssSparkConfig; import org.apache.spark.shuffle.ShuffleReader; import org.apache.spark.util.CompletionIterator; import org.apache.spark.util.CompletionIterator$; @@ -56,8 +57,6 @@ import org.apache.uniffle.common.config.RssClientConf; import org.apache.uniffle.common.config.RssConf; -import static org.apache.spark.shuffle.RssSparkConfig.RSS_RESUBMIT_STAGE_WITH_FETCH_FAILURE_ENABLED; - public class RssShuffleReader implements ShuffleReader { private static final Logger LOG = LoggerFactory.getLogger(RssShuffleReader.class); @@ -65,6 +64,7 @@ public class RssShuffleReader implements ShuffleReader { private String appId; private int shuffleId; + private int uniffleShuffleId; private int startPartition; private int endPartition; private TaskContext context; @@ -80,8 +80,10 @@ public class RssShuffleReader implements ShuffleReader { private Configuration hadoopConf; private RssConf rssConf; private Supplier managerClientSupplier; + private boolean rssStageRetryEnabled; public RssShuffleReader( + int uniffleShuffleId, int startPartition, int endPartition, TaskContext context, @@ -101,6 +103,7 @@ public RssShuffleReader( this.context = context; this.shuffleDependency = rssShuffleHandle.getDependency(); this.shuffleId = shuffleDependency.shuffleId(); + this.uniffleShuffleId = uniffleShuffleId; this.serializer = rssShuffleHandle.getDependency().serializer(); this.taskId = "" + context.taskAttemptId() + "_" + context.attemptNumber(); this.basePath = basePath; @@ -113,6 +116,7 @@ public RssShuffleReader( this.rssConf = rssConf; this.managerClientSupplier = managerClientSupplier; expectedTaskIdsBitmapFilterEnable = shuffleServerInfoList.size() > 1; + this.rssStageRetryEnabled = rssConf.getBoolean(RssSparkConfig.RSS_RESUBMIT_STAGE_ENABLED); } @Override @@ -131,7 +135,7 @@ public Iterator> read() { .createShuffleReadClient( ShuffleClientFactory.newReadBuilder() .appId(appId) - .shuffleId(shuffleId) + .shuffleId(uniffleShuffleId) .partitionId(startPartition) .basePath(basePath) .partitionNumPerRange(partitionNumPerRange) @@ -238,14 +242,16 @@ public BoxedUnit apply() { } // stage re-compute and shuffle manager server port are both set - if (rssConf.getBoolean(RSS_RESUBMIT_STAGE_WITH_FETCH_FAILURE_ENABLED) + if (rssStageRetryEnabled && rssConf.getInteger(RssClientConf.SHUFFLE_MANAGER_GRPC_PORT, 0) > 0) { resultIter = RssFetchFailedIterator.newBuilder() .appId(appId) .shuffleId(shuffleId) + .uniffleShuffleId(uniffleShuffleId) .partitionId(startPartition) - .stageAttemptId(context.stageAttemptNumber()) + .stageAttemptId(context.stageId()) + .stageAttemptNumber(context.stageAttemptNumber()) .managerClientSupplier(managerClientSupplier) .build(resultIter); } @@ -257,6 +263,8 @@ private String getReadInfo() { + appId + ", shuffleId=" + shuffleId + + ", uniffleShuffleId=" + + uniffleShuffleId + ",taskId=" + taskId + ", partitions: [" diff --git a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java index 1cd8113c0b..22f1c79ace 100644 --- a/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java +++ b/client-spark/spark2/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java @@ -39,7 +39,6 @@ import scala.collection.Iterator; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.common.util.concurrent.Uninterruptibles; @@ -50,11 +49,9 @@ import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; -import org.apache.spark.shuffle.FetchFailedException; import org.apache.spark.shuffle.RssShuffleHandle; import org.apache.spark.shuffle.RssShuffleManager; import org.apache.spark.shuffle.RssSparkConfig; -import org.apache.spark.shuffle.RssSparkShuffleUtils; import org.apache.spark.shuffle.ShuffleWriter; import org.apache.spark.shuffle.handle.ShuffleHandleInfo; import org.apache.spark.shuffle.handle.SimpleShuffleHandleInfo; @@ -64,9 +61,6 @@ import org.apache.uniffle.client.api.ShuffleManagerClient; import org.apache.uniffle.client.api.ShuffleWriteClient; -import org.apache.uniffle.client.impl.FailedBlockSendTracker; -import org.apache.uniffle.client.request.RssReportShuffleWriteFailureRequest; -import org.apache.uniffle.client.response.RssReportShuffleWriteFailureResponse; import org.apache.uniffle.common.ShuffleBlockInfo; import org.apache.uniffle.common.ShuffleServerInfo; import org.apache.uniffle.common.exception.RssException; @@ -74,8 +68,6 @@ import org.apache.uniffle.common.exception.RssWaitFailedException; import org.apache.uniffle.storage.util.StorageType; -import static org.apache.spark.shuffle.RssSparkConfig.RSS_RESUBMIT_STAGE_WITH_WRITE_FAILURE_ENABLED; - public class RssShuffleWriter extends ShuffleWriter { private static final Logger LOG = LoggerFactory.getLogger(RssShuffleWriter.class); @@ -91,6 +83,7 @@ public class RssShuffleWriter extends ShuffleWriter { private String appId; private int numMaps; private int shuffleId; + private int uniffleShuffleId; private final ShuffleHandleInfo shuffleHandleInfo; private int bitmapSplitNum; private String taskId; @@ -109,12 +102,11 @@ public class RssShuffleWriter extends ShuffleWriter { private TaskContext taskContext; private SparkConf sparkConf; private Supplier managerClientSupplier; - private boolean enableWriteFailureRetry; - private Set recordReportFailedShuffleservers; public RssShuffleWriter( String appId, int shuffleId, + int uniffleShuffleId, String taskId, long taskAttemptId, WriteBufferManager bufferManager, @@ -129,6 +121,7 @@ public RssShuffleWriter( this( appId, shuffleId, + uniffleShuffleId, taskId, taskAttemptId, shuffleWriteMetrics, @@ -146,6 +139,7 @@ public RssShuffleWriter( private RssShuffleWriter( String appId, int shuffleId, + int uniffleShuffleId, String taskId, long taskAttemptId, ShuffleWriteMetrics shuffleWriteMetrics, @@ -159,6 +153,7 @@ private RssShuffleWriter( TaskContext context) { this.appId = appId; this.shuffleId = shuffleId; + this.uniffleShuffleId = uniffleShuffleId; this.taskId = taskId; this.taskAttemptId = taskAttemptId; this.numMaps = rssHandle.getNumMaps(); @@ -181,14 +176,12 @@ private RssShuffleWriter( this.shuffleHandleInfo = shuffleHandleInfo; this.taskContext = context; this.sparkConf = sparkConf; - this.enableWriteFailureRetry = - RssSparkConfig.toRssConf(sparkConf).get(RSS_RESUBMIT_STAGE_WITH_WRITE_FAILURE_ENABLED); - this.recordReportFailedShuffleservers = Sets.newConcurrentHashSet(); } public RssShuffleWriter( String appId, int shuffleId, + int uniffleShuffleId, String taskId, long taskAttemptId, ShuffleWriteMetrics shuffleWriteMetrics, @@ -202,6 +195,7 @@ public RssShuffleWriter( this( appId, shuffleId, + uniffleShuffleId, taskId, taskAttemptId, shuffleWriteMetrics, @@ -211,13 +205,12 @@ public RssShuffleWriter( managerClientSupplier, rssHandle, taskFailureCallback, - shuffleManager.getShuffleHandleInfo( - context.stageId(), context.stageAttemptNumber(), rssHandle, true), + shuffleManager.getShuffleHandleInfo(uniffleShuffleId, rssHandle), context); BufferManagerOptions bufferOptions = new BufferManagerOptions(sparkConf); final WriteBufferManager bufferManager = new WriteBufferManager( - shuffleId, + uniffleShuffleId, taskId, taskAttemptId, bufferOptions, @@ -226,8 +219,7 @@ public RssShuffleWriter( context.taskMemoryManager(), shuffleWriteMetrics, RssSparkConfig.toRssConf(sparkConf), - this::processShuffleBlockInfos, - context.stageAttemptNumber()); + this::processShuffleBlockInfos); this.bufferManager = bufferManager; } @@ -249,11 +241,7 @@ public void write(Iterator> records) { writeImpl(records); } catch (Exception e) { taskFailureCallback.apply(taskId); - if (enableWriteFailureRetry) { - throwFetchFailedIfNecessary(e, Sets.newConcurrentHashSet()); - } else { - throw e; - } + throw e; } } @@ -297,6 +285,8 @@ private void writeImpl(Iterator> records) { + appId + "], shuffleId[" + shuffleId + + "], uniffleShuffleId[" + + uniffleShuffleId + "], taskId[" + taskId + "] with write " @@ -404,7 +394,9 @@ protected void sendCommit() { ExecutorService executor = Executors.newSingleThreadExecutor(); Future future = executor.submit( - () -> shuffleWriteClient.sendCommit(shuffleServersForData, appId, shuffleId, numMaps)); + () -> + shuffleWriteClient.sendCommit( + shuffleServersForData, appId, uniffleShuffleId, numMaps)); long start = System.currentTimeMillis(); int currentWait = 200; int maxWait = 5000; @@ -485,17 +477,12 @@ public Option stop(boolean success) { createDummyBlockManagerId(appId + "_" + taskId, taskAttemptId); long start = System.currentTimeMillis(); shuffleWriteClient.reportShuffleResult( - serverToPartitionToBlockIds, - appId, - shuffleId, - taskAttemptId, - bitmapSplitNum, - recordReportFailedShuffleservers, - enableWriteFailureRetry); + serverToPartitionToBlockIds, appId, uniffleShuffleId, taskAttemptId, bitmapSplitNum); long reportDuration = System.currentTimeMillis() - start; LOG.info( - "Reported all shuffle result for shuffleId[{}] task[{}] with bitmapNum[{}] cost {} ms", + "Reported all shuffle result for shuffleId[{}] uniffleShuffleId[{}] task[{}] with bitmapNum[{}] cost {} ms", shuffleId, + uniffleShuffleId, taskAttemptId, bitmapSplitNum, reportDuration); @@ -506,13 +493,7 @@ public Option stop(boolean success) { return Option.empty(); } } catch (Exception e) { - // If an exception is thrown during the reporting process, it should be judged as a failure - // and Stage retry should be triggered. - if (enableWriteFailureRetry) { - throw throwFetchFailedIfNecessary(e, recordReportFailedShuffleservers); - } else { - throw e; - } + throw e; } finally { // free all memory & metadata, or memory leak happen in executor if (bufferManager != null) { @@ -552,37 +533,4 @@ protected Map> getPartitionToBlockIds() { protected ShuffleWriteMetrics getShuffleWriteMetrics() { return shuffleWriteMetrics; } - - private RssException throwFetchFailedIfNecessary( - Exception e, Set reportFailuredServers) { - // The shuffleServer is registered only when a Block fails to be sent - if (e instanceof RssSendFailedException) { - FailedBlockSendTracker blockIdsFailedSendTracker = - shuffleManager.getBlockIdsFailedSendTracker(taskId); - List shuffleServerInfos = - Lists.newArrayList(blockIdsFailedSendTracker.getFaultyShuffleServers()); - shuffleServerInfos.addAll(reportFailuredServers); - RssReportShuffleWriteFailureRequest req = - new RssReportShuffleWriteFailureRequest( - appId, - shuffleId, - taskContext.stageId(), - taskContext.stageAttemptNumber(), - shuffleServerInfos, - e.getMessage()); - RssReportShuffleWriteFailureResponse response = - managerClientSupplier.get().reportShuffleWriteFailure(req); - if (response.getReSubmitWholeStage()) { - LOG.warn("Multiple task failures trigger Stage retry."); - // since we are going to roll out the whole stage, mapIndex shouldn't matter, hence -1 - // is - // provided. - FetchFailedException ffe = - RssSparkShuffleUtils.createFetchFailedException( - shuffleId, -1, taskContext.stageAttemptNumber(), e); - throw new RssException(ffe); - } - } - throw new RssException(e); - } } diff --git a/client-spark/spark2/src/test/java/org/apache/spark/shuffle/reader/RssShuffleReaderTest.java b/client-spark/spark2/src/test/java/org/apache/spark/shuffle/reader/RssShuffleReaderTest.java index 78fe7dec0d..bfeea6e285 100644 --- a/client-spark/spark2/src/test/java/org/apache/spark/shuffle/reader/RssShuffleReaderTest.java +++ b/client-spark/spark2/src/test/java/org/apache/spark/shuffle/reader/RssShuffleReaderTest.java @@ -91,6 +91,7 @@ public void readTest() throws Exception { RssShuffleReader rssShuffleReaderSpy = spy( new RssShuffleReader<>( + 0, 0, 1, contextMock, diff --git a/client-spark/spark2/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java b/client-spark/spark2/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java index 331697b851..68e8b8a222 100644 --- a/client-spark/spark2/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java +++ b/client-spark/spark2/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java @@ -120,6 +120,7 @@ public void checkBlockSendResultTest() { new RssShuffleWriter<>( "appId", 0, + 0, taskId, 1L, bufferManagerSpy, @@ -287,8 +288,7 @@ public void writeTest() throws Exception { mockTaskMemoryManager, shuffleWriteMetrics, new RssConf(), - null, - 0); + null); WriteBufferManager bufferManagerSpy = spy(bufferManager); doReturn(1000000L).when(bufferManagerSpy).acquireMemory(anyLong()); TaskContext contextMock = mock(TaskContext.class); @@ -298,6 +298,7 @@ public void writeTest() throws Exception { new RssShuffleWriter<>( "appId", 0, + 0, "taskId", 1L, bufferManagerSpy, @@ -412,6 +413,7 @@ public void postBlockEventTest() throws Exception { new RssShuffleWriter<>( "appId", 0, + 0, "taskId", 1L, bufferManager, diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java index c6569ec91c..cd3db44f5c 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/RssShuffleManager.java @@ -42,11 +42,11 @@ import org.apache.spark.broadcast.Broadcast; import org.apache.spark.executor.ShuffleReadMetrics; import org.apache.spark.executor.ShuffleWriteMetrics; +import org.apache.spark.rdd.DeterministicLevel; import org.apache.spark.shuffle.events.ShuffleAssignmentInfoEvent; import org.apache.spark.shuffle.handle.MutableShuffleHandleInfo; import org.apache.spark.shuffle.handle.ShuffleHandleInfo; import org.apache.spark.shuffle.handle.SimpleShuffleHandleInfo; -import org.apache.spark.shuffle.handle.StageAttemptShuffleHandleInfo; import org.apache.spark.shuffle.reader.RssShuffleReader; import org.apache.spark.shuffle.writer.DataPusher; import org.apache.spark.shuffle.writer.RssShuffleWriter; @@ -134,16 +134,21 @@ public ShuffleHandle registerShuffle( dataPusher.setRssAppId(id.get()); } LOG.info("Generate application id used in rss: " + id.get()); + // If stage retry is enabled, the Deterministic status of the ShuffleId needs to be recorded. + if (rssStageRetryEnabled) { + shuffleIdMappingManager.recordShuffleIdDeterminate( + shuffleId, + dependency.rdd().getOutputDeterministicLevel() != DeterministicLevel.INDETERMINATE()); + } if (dependency.partitioner().numPartitions() == 0) { shuffleIdToPartitionNum.putIfAbsent(shuffleId, 0); shuffleIdToNumMapTasks.computeIfAbsent( shuffleId, key -> dependency.rdd().partitions().length); LOG.info( - "RegisterShuffle with ShuffleId[" - + shuffleId - + "], partitionNum is 0, " - + "return the empty RssShuffleHandle directly"); + "RegisterShuffle with ShuffleId[{}], partitionNum is 0, " + + "return the empty RssShuffleHandle directly", + shuffleId); Broadcast hdlInfoBd = RssSparkShuffleUtils.broadcastShuffleHdlInfo( RssSparkShuffleUtils.getActiveSparkContext(), @@ -154,72 +159,68 @@ public ShuffleHandle registerShuffle( shuffleId, id.get(), dependency.rdd().getNumPartitions(), dependency, hdlInfoBd); } - String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key()); - RemoteStorageInfo defaultRemoteStorage = getDefaultRemoteStorageInfo(sparkConf); - RemoteStorageInfo remoteStorage = - ClientUtils.fetchRemoteStorage( - id.get(), defaultRemoteStorage, dynamicConfEnabled, storageType, shuffleWriteClient); - Set assignmentTags = RssSparkShuffleUtils.getAssignmentTags(sparkConf); ClientUtils.validateClientType(clientType); assignmentTags.add(clientType); - int requiredShuffleServerNumber = RssSparkShuffleUtils.getRequiredShuffleServerNumber(sparkConf); int estimateTaskConcurrency = RssSparkShuffleUtils.estimateTaskConcurrency(sparkConf); - Map> partitionToServers = - requestShuffleAssignment( - shuffleId, - dependency.partitioner().numPartitions(), - 1, - requiredShuffleServerNumber, - estimateTaskConcurrency, - rssStageResubmitManager.getServerIdBlackList(), - 0); + Map> partitionToServers = null; + String storageType = sparkConf.get(RssSparkConfig.RSS_STORAGE_TYPE.key()); + RemoteStorageInfo defaultRemoteStorage = getDefaultRemoteStorageInfo(sparkConf); + RemoteStorageInfo remoteStorage = + ClientUtils.fetchRemoteStorage( + id.get(), defaultRemoteStorage, dynamicConfEnabled, storageType, shuffleWriteClient); + if (!rssStageRetryEnabled) { + // The ShuffleServer will be allocated here only when the Spark phase retry is not enabled; + // otherwise, the ShuffleServer will be allocated and registered during the Shuffle write + // phase. + partitionToServers = + requestShuffleAssignment( + shuffleId, + dependency.partitioner().numPartitions(), + 1, + requiredShuffleServerNumber, + estimateTaskConcurrency, + rssStageResubmitManager.getServerIdBlackList()); + if (partitionReassignEnabled) { + // Driver management is enabled only when Partition reallocation is enabled alone. + MutableShuffleHandleInfo shuffleHandleInfo = + new MutableShuffleHandleInfo( + shuffleId, partitionToServers, remoteStorage, partitionSplitMode); + shuffleHandleInfoManager.register(shuffleId, shuffleHandleInfo); + } + } startHeartbeat(); shuffleIdToPartitionNum.computeIfAbsent( shuffleId, key -> dependency.partitioner().numPartitions()); shuffleIdToNumMapTasks.computeIfAbsent(shuffleId, key -> dependency.rdd().partitions().length); - if (shuffleManagerRpcServiceEnabled && rssStageRetryForWriteFailureEnabled) { - ShuffleHandleInfo shuffleHandleInfo = - new MutableShuffleHandleInfo( - shuffleId, partitionToServers, remoteStorage, partitionSplitMode); - StageAttemptShuffleHandleInfo handleInfo = - new StageAttemptShuffleHandleInfo(shuffleId, remoteStorage, shuffleHandleInfo); - shuffleHandleInfoManager.register(shuffleId, handleInfo); - } else if (shuffleManagerRpcServiceEnabled && partitionReassignEnabled) { - ShuffleHandleInfo shuffleHandleInfo = - new MutableShuffleHandleInfo( - shuffleId, partitionToServers, remoteStorage, partitionSplitMode); - shuffleHandleInfoManager.register(shuffleId, shuffleHandleInfo); - } + Broadcast hdlInfoBd = RssSparkShuffleUtils.broadcastShuffleHdlInfo( RssSparkShuffleUtils.getActiveSparkContext(), shuffleId, partitionToServers, remoteStorage); - LOG.info( - "RegisterShuffle with ShuffleId[" - + shuffleId - + "], partitionNum[" - + partitionToServers.size() - + "], shuffleServerForResult: " - + partitionToServers); - - // Post assignment event - RssSparkShuffleUtils.getActiveSparkContext() - .listenerBus() - .post( - new ShuffleAssignmentInfoEvent( - shuffleId, - new ArrayList<>( - partitionToServers.values().stream() - .flatMap(x -> x.stream()) - .map(x -> x.getId()) - .collect(Collectors.toSet())))); - + if (partitionToServers != null) { + LOG.info( + "RegisterShuffle with ShuffleId[{}], partitionNum[{}], shuffleServerForResult: {}", + shuffleId, + partitionToServers.size(), + partitionToServers); + // Post assignment event + RssSparkShuffleUtils.getActiveSparkContext() + .listenerBus() + .post( + new ShuffleAssignmentInfoEvent( + shuffleId, + new ArrayList<>( + partitionToServers.values().stream() + .flatMap(x -> x.stream()) + .map(x -> x.getId()) + .collect(Collectors.toSet())))); + } return new RssShuffleHandle<>( shuffleId, id.get(), dependency.rdd().getNumPartitions(), dependency, hdlInfoBd); } @@ -233,6 +234,12 @@ public ShuffleWriter getWriter( RssShuffleHandle rssHandle = (RssShuffleHandle) handle; setPusherAppId(rssHandle); int shuffleId = rssHandle.getShuffleId(); + int uniffleShuffleId; + if (rssStageRetryEnabled) { + uniffleShuffleId = getUniffleShuffleId(shuffleId, context, true); + } else { + uniffleShuffleId = shuffleId; + } ShuffleWriteMetrics writeMetrics; if (metrics != null) { writeMetrics = new WriteMetrics(metrics); @@ -243,6 +250,7 @@ public ShuffleWriter getWriter( return new RssShuffleWriter<>( rssHandle.getAppId(), shuffleId, + uniffleShuffleId, taskId, getTaskAttemptIdForBlockId(context.partitionId(), context.attemptNumber()), writeMetrics, @@ -360,24 +368,14 @@ public ShuffleReader getReaderImpl( RssShuffleHandle rssShuffleHandle = (RssShuffleHandle) handle; final int partitionNum = rssShuffleHandle.getDependency().partitioner().numPartitions(); int shuffleId = rssShuffleHandle.getShuffleId(); - ShuffleHandleInfo shuffleHandleInfo; - if (shuffleManagerRpcServiceEnabled && rssStageRetryForWriteFailureEnabled) { - // In Stage Retry mode, Get the ShuffleServer list from the Driver based on the shuffleId. - shuffleHandleInfo = - getRemoteShuffleHandleInfoWithStageRetry( - context.stageId(), context.stageAttemptNumber(), shuffleId, false); - } else if (shuffleManagerRpcServiceEnabled && partitionReassignEnabled) { - // In Block Retry mode, Get the ShuffleServer list from the Driver based on the shuffleId. - shuffleHandleInfo = - getRemoteShuffleHandleInfoWithBlockRetry( - context.stageId(), context.stageAttemptNumber(), shuffleId, false); + int uniffleShuffleId; + if (rssStageRetryEnabled) { + uniffleShuffleId = getUniffleShuffleId(shuffleId, context, false); } else { - shuffleHandleInfo = - new SimpleShuffleHandleInfo( - shuffleId, - rssShuffleHandle.getPartitionToServers(), - rssShuffleHandle.getRemoteStorage()); + uniffleShuffleId = shuffleId; } + ShuffleHandleInfo shuffleHandleInfo; + shuffleHandleInfo = getShuffleHandleInfo(uniffleShuffleId, rssShuffleHandle); Map> serverToPartitions = getPartitionDataServers(shuffleHandleInfo, startPartition, endPartition); long start = System.currentTimeMillis(); @@ -387,20 +385,17 @@ public ShuffleReader getReaderImpl( serverToPartitions, rssShuffleHandle.getAppId(), shuffleId, + uniffleShuffleId, + context.stageId(), context.stageAttemptNumber(), shuffleHandleInfo.createPartitionReplicaTracking()); LOG.info( - "Get shuffle blockId cost " - + (System.currentTimeMillis() - start) - + " ms, and get " - + blockIdBitmap.getLongCardinality() - + " blockIds for shuffleId[" - + shuffleId - + "], startPartition[" - + startPartition - + "], endPartition[" - + endPartition - + "]"); + "Get shuffle blockId cost {} ms, and get {} blockIds for shuffleId[{}], startPartition[{}], endPartition[{}]", + (System.currentTimeMillis() - start), + blockIdBitmap.getLongCardinality(), + shuffleId, + startPartition, + endPartition); ShuffleReadMetrics readMetrics; if (metrics != null) { @@ -416,6 +411,7 @@ public ShuffleReader getReaderImpl( RssSparkShuffleUtils.getRemoteStorageHadoopConf(sparkConf, shuffleRemoteStorageInfo); return new RssShuffleReader( + uniffleShuffleId, startPartition, endPartition, startMapIndex, @@ -666,7 +662,9 @@ private Roaring64NavigableMap getShuffleResultForMultiPart( Map> serverToPartitions, String appId, int shuffleId, + int uniffleShuffleId, int stageAttemptId, + int stageAttemptNumber, PartitionDataReplicaRequirementTracking replicaRequirementTracking) { Set failedPartitions = Sets.newHashSet(); try { @@ -674,12 +672,20 @@ private Roaring64NavigableMap getShuffleResultForMultiPart( clientType, serverToPartitions, appId, - shuffleId, + uniffleShuffleId, failedPartitions, replicaRequirementTracking); } catch (RssFetchFailedException e) { throw RssSparkShuffleUtils.reportRssFetchFailedException( - managerClientSupplier, e, sparkConf, appId, shuffleId, stageAttemptId, failedPartitions); + managerClientSupplier, + e, + sparkConf, + appId, + shuffleId, + uniffleShuffleId, + stageAttemptId, + stageAttemptNumber, + failedPartitions); } } } diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/reader/RssShuffleReader.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/reader/RssShuffleReader.java index a1c345e848..ada2f575cc 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/reader/RssShuffleReader.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/reader/RssShuffleReader.java @@ -43,6 +43,7 @@ import org.apache.spark.serializer.Serializer; import org.apache.spark.shuffle.FunctionUtils; import org.apache.spark.shuffle.RssShuffleHandle; +import org.apache.spark.shuffle.RssSparkConfig; import org.apache.spark.shuffle.ShuffleReader; import org.apache.spark.util.CompletionIterator; import org.apache.spark.util.CompletionIterator$; @@ -64,14 +65,13 @@ import org.apache.uniffle.common.rpc.StatusCode; import org.apache.uniffle.storage.handler.impl.ShuffleServerReadCostTracker; -import static org.apache.spark.shuffle.RssSparkConfig.RSS_RESUBMIT_STAGE_WITH_FETCH_FAILURE_ENABLED; - public class RssShuffleReader implements ShuffleReader { private static final Logger LOG = LoggerFactory.getLogger(RssShuffleReader.class); private final Map> partitionToShuffleServers; private String appId; private int shuffleId; + private int uniffleShuffleId; private int startPartition; private int endPartition; private TaskContext context; @@ -92,8 +92,10 @@ public class RssShuffleReader implements ShuffleReader { private Supplier managerClientSupplier; private ShuffleServerReadCostTracker shuffleServerReadCostTracker = new ShuffleServerReadCostTracker(); + private boolean rssStageRetryEnabled; public RssShuffleReader( + int uniffleShuffleId, int startPartition, int endPartition, int mapStartIndex, @@ -119,6 +121,7 @@ public RssShuffleReader( this.numMaps = rssShuffleHandle.getNumMaps(); this.shuffleDependency = rssShuffleHandle.getDependency(); this.shuffleId = shuffleDependency.shuffleId(); + this.uniffleShuffleId = uniffleShuffleId; this.serializer = rssShuffleHandle.getDependency().serializer(); this.taskId = "" + context.taskAttemptId() + "_" + context.attemptNumber(); this.basePath = basePath; @@ -131,6 +134,7 @@ public RssShuffleReader( this.rssConf = rssConf; this.dataDistributionType = dataDistributionType; this.managerClientSupplier = managerClientSupplier; + this.rssStageRetryEnabled = rssConf.getBoolean(RssSparkConfig.RSS_RESUBMIT_STAGE_ENABLED); } @Override @@ -202,14 +206,16 @@ public Void apply(TaskContext context) { resultIter = new InterruptibleIterator<>(context, resultIter); } // resubmit stage and shuffle manager server port are both set - if (rssConf.getBoolean(RSS_RESUBMIT_STAGE_WITH_FETCH_FAILURE_ENABLED) + if (rssStageRetryEnabled && rssConf.getInteger(RssClientConf.SHUFFLE_MANAGER_GRPC_PORT, 0) > 0) { resultIter = RssFetchFailedIterator.newBuilder() .appId(appId) .shuffleId(shuffleId) + .uniffleShuffleId(uniffleShuffleId) .partitionId(startPartition) - .stageAttemptId(context.stageAttemptNumber()) + .stageAttemptId(context.stageId()) + .stageAttemptNumber(context.stageAttemptNumber()) .managerClientSupplier(managerClientSupplier) .build(resultIter); } @@ -221,6 +227,8 @@ private String getReadInfo() { + appId + ", shuffleId=" + shuffleId + + ", uniffleShuffleId=" + + uniffleShuffleId + ",taskId=" + taskId + ", partitions: [" @@ -275,7 +283,7 @@ class MultiPartitionIterator extends AbstractIterator> { ShuffleClientFactory.newReadBuilder() .readCostTracker(shuffleServerReadCostTracker) .appId(appId) - .shuffleId(shuffleId) + .shuffleId(uniffleShuffleId) .partitionId(partition) .basePath(basePath) .partitionNumPerRange(1) @@ -347,7 +355,7 @@ private void postShuffleReadMetricsToDriver() { client.reportShuffleReadMetric( new RssReportShuffleReadMetricRequest( context.stageId(), - shuffleId, + uniffleShuffleId, context.taskAttemptId(), shuffleServerReadCostTracker.list().entrySet().stream() .collect( diff --git a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java index a4e538e009..776b91c901 100644 --- a/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java +++ b/client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java @@ -56,11 +56,9 @@ import org.apache.spark.TaskContext; import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.scheduler.MapStatus; -import org.apache.spark.shuffle.FetchFailedException; import org.apache.spark.shuffle.RssShuffleHandle; import org.apache.spark.shuffle.RssShuffleManager; import org.apache.spark.shuffle.RssSparkConfig; -import org.apache.spark.shuffle.RssSparkShuffleUtils; import org.apache.spark.shuffle.ShuffleWriter; import org.apache.spark.shuffle.handle.MutableShuffleHandleInfo; import org.apache.spark.shuffle.handle.ShuffleHandleInfo; @@ -73,10 +71,8 @@ import org.apache.uniffle.client.impl.FailedBlockSendTracker; import org.apache.uniffle.client.impl.TrackingBlockStatus; import org.apache.uniffle.client.request.RssReassignOnBlockSendFailureRequest; -import org.apache.uniffle.client.request.RssReportShuffleWriteFailureRequest; import org.apache.uniffle.client.request.RssReportShuffleWriteMetricRequest; import org.apache.uniffle.client.response.RssReassignOnBlockSendFailureResponse; -import org.apache.uniffle.client.response.RssReportShuffleWriteFailureResponse; import org.apache.uniffle.client.response.RssReportShuffleWriteMetricResponse; import org.apache.uniffle.common.ReceivingFailureServer; import org.apache.uniffle.common.ShuffleBlockInfo; @@ -90,7 +86,6 @@ import static org.apache.spark.shuffle.RssSparkConfig.RSS_CLIENT_MAP_SIDE_COMBINE_ENABLED; import static org.apache.spark.shuffle.RssSparkConfig.RSS_PARTITION_REASSIGN_BLOCK_RETRY_MAX_TIMES; -import static org.apache.spark.shuffle.RssSparkConfig.RSS_RESUBMIT_STAGE_WITH_WRITE_FAILURE_ENABLED; public class RssShuffleWriter extends ShuffleWriter { @@ -101,6 +96,7 @@ public class RssShuffleWriter extends ShuffleWriter { private final String appId; private final int shuffleId; + private final int uniffleShuffleId; private final ShuffleHandleInfo shuffleHandleInfo; private WriteBufferManager bufferManager; private String taskId; @@ -140,14 +136,13 @@ public class RssShuffleWriter extends ShuffleWriter { Sets.newHashSet(StatusCode.NO_REGISTER); private final Supplier managerClientSupplier; - private boolean enableWriteFailureRetry; - private Set recordReportFailedShuffleservers; // Only for tests @VisibleForTesting public RssShuffleWriter( String appId, int shuffleId, + int uniffleShuffleId, String taskId, long taskAttemptId, WriteBufferManager bufferManager, @@ -162,6 +157,7 @@ public RssShuffleWriter( this( appId, shuffleId, + uniffleShuffleId, taskId, taskAttemptId, shuffleWriteMetrics, @@ -180,6 +176,7 @@ public RssShuffleWriter( private RssShuffleWriter( String appId, int shuffleId, + int uniffleShuffleId, String taskId, long taskAttemptId, ShuffleWriteMetrics shuffleWriteMetrics, @@ -192,13 +189,15 @@ private RssShuffleWriter( ShuffleHandleInfo shuffleHandleInfo, TaskContext context) { LOG.info( - "RssShuffle start write taskAttemptId[{}] data with RssHandle[appId {}, shuffleId {}].", + "RssShuffle start write taskAttemptId[{}] data with RssHandle[appId {}, shuffleId {}, uniffleShuffleId {}].", taskAttemptId, rssHandle.getAppId(), - rssHandle.getShuffleId()); + rssHandle.getShuffleId(), + uniffleShuffleId); this.shuffleManager = shuffleManager; this.appId = appId; this.shuffleId = shuffleId; + this.uniffleShuffleId = uniffleShuffleId; this.taskId = taskId; this.taskAttemptId = taskAttemptId; this.numMaps = rssHandle.getNumMaps(); @@ -228,15 +227,13 @@ private RssShuffleWriter( RssClientConf.RSS_CLIENT_REASSIGN_ENABLED.defaultValue()); this.blockFailSentRetryMaxTimes = RssSparkConfig.toRssConf(sparkConf).get(RSS_PARTITION_REASSIGN_BLOCK_RETRY_MAX_TIMES); - this.enableWriteFailureRetry = - RssSparkConfig.toRssConf(sparkConf).get(RSS_RESUBMIT_STAGE_WITH_WRITE_FAILURE_ENABLED); - this.recordReportFailedShuffleservers = Sets.newConcurrentHashSet(); } // Gluten needs this method public RssShuffleWriter( String appId, int shuffleId, + int uniffleShuffleId, String taskId, long taskAttemptId, ShuffleWriteMetrics shuffleWriteMetrics, @@ -249,6 +246,7 @@ public RssShuffleWriter( this( appId, shuffleId, + uniffleShuffleId, taskId, taskAttemptId, shuffleWriteMetrics, @@ -264,6 +262,7 @@ public RssShuffleWriter( public RssShuffleWriter( String appId, int shuffleId, + int uniffleShuffleId, String taskId, long taskAttemptId, ShuffleWriteMetrics shuffleWriteMetrics, @@ -277,6 +276,7 @@ public RssShuffleWriter( this( appId, shuffleId, + uniffleShuffleId, taskId, taskAttemptId, shuffleWriteMetrics, @@ -286,14 +286,13 @@ public RssShuffleWriter( managerClientSupplier, rssHandle, taskFailureCallback, - shuffleManager.getShuffleHandleInfo( - context.stageId(), context.stageAttemptNumber(), rssHandle, true), + shuffleManager.getShuffleHandleInfo(uniffleShuffleId, rssHandle), context); this.taskAttemptAssignment = new TaskAttemptAssignment(taskAttemptId, shuffleHandleInfo); BufferManagerOptions bufferOptions = new BufferManagerOptions(sparkConf); final WriteBufferManager bufferManager = new WriteBufferManager( - shuffleId, + uniffleShuffleId, taskId, taskAttemptId, bufferOptions, @@ -302,8 +301,7 @@ public RssShuffleWriter( shuffleWriteMetrics, RssSparkConfig.toRssConf(sparkConf), this::processShuffleBlockInfos, - this::getPartitionAssignedServers, - context.stageAttemptNumber()); + this::getPartitionAssignedServers); this.bufferManager = bufferManager; } @@ -322,11 +320,7 @@ public void write(Iterator> records) { writeImpl(records); } catch (Exception e) { taskFailureCallback.apply(taskId); - if (enableWriteFailureRetry) { - throwFetchFailedIfNecessary(e, Sets.newConcurrentHashSet()); - } else { - throw e; - } + throw e; } } @@ -379,20 +373,16 @@ protected void writeImpl(Iterator> records) { long writeDurationMs = bufferManager.getWriteTime() + (System.currentTimeMillis() - start); shuffleWriteMetrics.incWriteTime(TimeUnit.MILLISECONDS.toNanos(writeDurationMs)); LOG.info( - "Finish write shuffle for appId[" - + appId - + "], shuffleId[" - + shuffleId - + "], taskId[" - + taskId - + "] with write " - + writeDurationMs - + " ms, include checkSendResult[" - + checkDuration - + "], commit[" - + (System.currentTimeMillis() - commitStartTs) - + "], " - + bufferManager.getManagerCostInfo()); + "Finish write shuffle for appId[{}], shuffleId[{}], uniffleShuffleId[{}], taskId[{}] with write {} ms," + + " include checkSendResult[{}], commit[{}], {}", + appId, + shuffleId, + uniffleShuffleId, + taskId, + writeDurationMs, + checkDuration, + (System.currentTimeMillis() - commitStartTs), + bufferManager.getManagerCostInfo()); } private void checkAllBufferSpilled() { @@ -734,7 +724,7 @@ private void doReassignOnBlockSendFailure( try { RssReassignOnBlockSendFailureRequest request = new RssReassignOnBlockSendFailureRequest( - shuffleId, + uniffleShuffleId, failurePartitionToServers, executorId, taskAttemptId, @@ -847,7 +837,9 @@ protected void sendCommit() { ExecutorService executor = Executors.newSingleThreadExecutor(); Future future = executor.submit( - () -> shuffleWriteClient.sendCommit(shuffleServersForData, appId, shuffleId, numMaps)); + () -> + shuffleWriteClient.sendCommit( + shuffleServersForData, appId, uniffleShuffleId, numMaps)); int maxWait = 5000; int currentWait = 200; long start = System.currentTimeMillis(); @@ -889,17 +881,12 @@ public Option stop(boolean success) { if (success) { long start = System.currentTimeMillis(); shuffleWriteClient.reportShuffleResult( - serverToPartitionToBlockIds, - appId, - shuffleId, - taskAttemptId, - bitmapSplitNum, - recordReportFailedShuffleservers, - enableWriteFailureRetry); + serverToPartitionToBlockIds, appId, uniffleShuffleId, taskAttemptId, bitmapSplitNum); long reportDuration = System.currentTimeMillis() - start; LOG.info( - "Reported all shuffle result for shuffleId[{}] task[{}] with bitmapNum[{}] cost {} ms", + "Reported all shuffle result for shuffleId[{}] uniffleShuffleId[{}] task[{}] with bitmapNum[{}] cost {} ms", shuffleId, + uniffleShuffleId, taskAttemptId, bitmapSplitNum, reportDuration); @@ -918,13 +905,7 @@ public Option stop(boolean success) { return Option.empty(); } } catch (Exception e) { - // If an exception is thrown during the reporting process, it should be judged as a failure - // and Stage retry should be triggered. - if (enableWriteFailureRetry) { - throw throwFetchFailedIfNecessary(e, recordReportFailedShuffleservers); - } else { - throw e; - } + throw e; } finally { // report shuffle write metrics to driver if (managerClientSupplier != null) { @@ -934,7 +915,7 @@ public Option stop(boolean success) { shuffleManagerClient.reportShuffleWriteMetric( new RssReportShuffleWriteMetricRequest( taskContext.stageId(), - shuffleId, + uniffleShuffleId, taskContext.taskAttemptId(), bufferManager.getShuffleServerPushCostTracker().toMetric())); if (response.getStatusCode() != StatusCode.SUCCESS) { @@ -984,36 +965,6 @@ public WriteBufferManager getBufferManager() { return bufferManager; } - private RssException throwFetchFailedIfNecessary( - Exception e, Set reportFailuredServers) { - // The shuffleServer is registered only when a Block fails to be sent - if (e instanceof RssSendFailedException) { - FailedBlockSendTracker blockIdsFailedSendTracker = - shuffleManager.getBlockIdsFailedSendTracker(taskId); - List shuffleServerInfos = - Lists.newArrayList(blockIdsFailedSendTracker.getFaultyShuffleServers()); - shuffleServerInfos.addAll(reportFailuredServers); - RssReportShuffleWriteFailureRequest req = - new RssReportShuffleWriteFailureRequest( - appId, - shuffleId, - taskContext.stageId(), - taskContext.stageAttemptNumber(), - shuffleServerInfos, - e.getMessage()); - RssReportShuffleWriteFailureResponse response = - managerClientSupplier.get().reportShuffleWriteFailure(req); - if (response.getReSubmitWholeStage()) { - LOG.warn(response.getMessage()); - FetchFailedException ffe = - RssSparkShuffleUtils.createFetchFailedException( - shuffleId, -1, taskContext.stageAttemptNumber(), e); - throw new RssException(ffe); - } - } - throw new RssException(e); - } - @VisibleForTesting protected void enableBlockFailSentRetry() { this.blockFailSentRetryEnabled = true; diff --git a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/RssShuffleManagerTest.java b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/RssShuffleManagerTest.java index 2a92b6ed52..19e25113f0 100644 --- a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/RssShuffleManagerTest.java +++ b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/RssShuffleManagerTest.java @@ -27,14 +27,12 @@ import org.apache.uniffle.client.util.RssClientConfig; import org.apache.uniffle.common.ShuffleDataDistributionType; -import org.apache.uniffle.common.config.ConfigOption; import org.apache.uniffle.common.config.RssClientConf; import org.apache.uniffle.common.exception.RssException; import org.apache.uniffle.common.rpc.StatusCode; import org.apache.uniffle.common.util.BlockIdLayout; import org.apache.uniffle.storage.util.StorageType; -import static org.apache.spark.shuffle.RssSparkConfig.RSS_RESUBMIT_STAGE_WITH_FETCH_FAILURE_ENABLED; import static org.apache.spark.shuffle.RssSparkConfig.RSS_SHUFFLE_MANAGER_GRPC_PORT; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -111,8 +109,6 @@ public void testCreateShuffleManagerServer() { RssShuffleManager shuffleManager = new RssShuffleManager(conf, true); - ConfigOption a = RSS_RESUBMIT_STAGE_WITH_FETCH_FAILURE_ENABLED; - assertTrue(conf.get(RSS_SHUFFLE_MANAGER_GRPC_PORT) > 0); } @@ -183,8 +179,6 @@ public void testWithStageRetry() { SparkConf conf = createSparkConf(); RssShuffleManager shuffleManager = new RssShuffleManager(conf, true); assertFalse(shuffleManager.isRssStageRetryEnabled()); - assertFalse(shuffleManager.isRssStageRetryForFetchFailureEnabled()); - assertFalse(shuffleManager.isRssStageRetryForWriteFailureEnabled()); shuffleManager.stop(); // case2: enable the stage retry @@ -193,31 +187,6 @@ public void testWithStageRetry() { "true"); shuffleManager = new RssShuffleManager(conf, true); assertTrue(shuffleManager.isRssStageRetryEnabled()); - assertTrue(shuffleManager.isRssStageRetryForFetchFailureEnabled()); - assertTrue(shuffleManager.isRssStageRetryForWriteFailureEnabled()); - shuffleManager.stop(); - - // case3: overwrite the stage retry - conf.set( - RssSparkConfig.SPARK_RSS_CONFIG_PREFIX - + RSS_RESUBMIT_STAGE_WITH_FETCH_FAILURE_ENABLED.key(), - "false"); - shuffleManager = new RssShuffleManager(conf, true); - assertTrue(shuffleManager.isRssStageRetryEnabled()); - assertFalse(shuffleManager.isRssStageRetryForFetchFailureEnabled()); - assertTrue(shuffleManager.isRssStageRetryForWriteFailureEnabled()); - shuffleManager.stop(); - - // case4: enable the partial stage retry of fetch failure - conf = createSparkConf(); - conf.set( - RssSparkConfig.SPARK_RSS_CONFIG_PREFIX - + RSS_RESUBMIT_STAGE_WITH_FETCH_FAILURE_ENABLED.key(), - "true"); - shuffleManager = new RssShuffleManager(conf, true); - assertTrue(shuffleManager.isRssStageRetryEnabled()); - assertTrue(shuffleManager.isRssStageRetryForFetchFailureEnabled()); - assertFalse(shuffleManager.isRssStageRetryForWriteFailureEnabled()); shuffleManager.stop(); } diff --git a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/reader/RssShuffleReaderTest.java b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/reader/RssShuffleReaderTest.java index bc77f71920..ca6b60c057 100644 --- a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/reader/RssShuffleReaderTest.java +++ b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/reader/RssShuffleReaderTest.java @@ -99,6 +99,7 @@ public void readTest() throws Exception { RssShuffleReader rssShuffleReaderSpy = spy( new RssShuffleReader<>( + 0, 0, 1, 0, @@ -123,6 +124,7 @@ public void readTest() throws Exception { RssShuffleReader rssShuffleReaderSpy1 = spy( new RssShuffleReader<>( + 0, 0, 2, 0, @@ -144,6 +146,7 @@ public void readTest() throws Exception { RssShuffleReader rssShuffleReaderSpy2 = spy( new RssShuffleReader<>( + 0, 0, 2, 0, diff --git a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java index 9e9e68db52..a9a25efea4 100644 --- a/client-spark/spark3/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java +++ b/client-spark/spark3/src/test/java/org/apache/spark/shuffle/writer/RssShuffleWriterTest.java @@ -175,6 +175,7 @@ private RssShuffleWriter createMockWriter(MutableShuffleHandleInfo shuffleHandle new RssShuffleWriter<>( "appId", 0, + 0, taskId, 1L, bufferManagerSpy, @@ -448,6 +449,7 @@ public void blockFailureResendTest() { new RssShuffleWriter<>( "appId", 0, + 0, "taskId", 1L, bufferManagerSpy, @@ -587,6 +589,7 @@ public void checkBlockSendResultTest() { new RssShuffleWriter<>( "appId", 0, + 0, "taskId", 1L, bufferManagerSpy, @@ -717,8 +720,7 @@ public void dataConsistencyWhenSpillTriggeredTest() throws Exception { fakedTaskMemoryManager, new ShuffleWriteMetrics(), RssSparkConfig.toRssConf(conf), - null, - 0); + null); Serializer kryoSerializer = new KryoSerializer(conf); Partitioner mockPartitioner = mock(Partitioner.class); @@ -737,6 +739,7 @@ public void dataConsistencyWhenSpillTriggeredTest() throws Exception { new RssShuffleWriter<>( "appId", 0, + 0, "taskId", 1L, bufferManager, @@ -862,6 +865,7 @@ public void writeTest() throws Exception { new RssShuffleWriter<>( "appId", 0, + 0, "taskId", 1L, bufferManagerSpy, @@ -978,6 +982,7 @@ public void postBlockEventTest() throws Exception { new RssShuffleWriter<>( "appId", 0, + 0, "taskId", 1L, bufferManagerSpy, diff --git a/client-tez/src/main/java/org/apache/tez/dag/app/TezRemoteShuffleManager.java b/client-tez/src/main/java/org/apache/tez/dag/app/TezRemoteShuffleManager.java index f44ad0c5e2..d9b3881cdf 100644 --- a/client-tez/src/main/java/org/apache/tez/dag/app/TezRemoteShuffleManager.java +++ b/client-tez/src/main/java/org/apache/tez/dag/app/TezRemoteShuffleManager.java @@ -305,7 +305,6 @@ public ShuffleAssignmentsInfo run() throws Exception { ShuffleDataDistributionType.NORMAL, RssTezConfig.toRssConf(conf) .get(MAX_CONCURRENCY_PER_PARTITION_TO_WRITE), - 0, StringUtils.isBlank(keyClassName) ? null : RssProtos.MergeContext.newBuilder() diff --git a/client-tez/src/test/java/org/apache/tez/runtime/library/common/sort/buffer/WriteBufferManagerTest.java b/client-tez/src/test/java/org/apache/tez/runtime/library/common/sort/buffer/WriteBufferManagerTest.java index 05fd55bc2e..71083699be 100644 --- a/client-tez/src/test/java/org/apache/tez/runtime/library/common/sort/buffer/WriteBufferManagerTest.java +++ b/client-tez/src/test/java/org/apache/tez/runtime/library/common/sort/buffer/WriteBufferManagerTest.java @@ -715,7 +715,6 @@ public void registerShuffle( RemoteStorageInfo remoteStorage, ShuffleDataDistributionType dataDistributionType, int maxConcurrencyPerPartitionToWrite, - int stageAttemptNumber, RssProtos.MergeContext mergeContext, Map properties) {} @@ -775,9 +774,6 @@ public ShuffleAssignmentsInfo getShuffleAssignments( int assignmentShuffleServerNumber, int estimateTaskConcurrency, Set faultyServerIds, - int stageId, - int stageAttemptNumber, - boolean reassign, long retryIntervalMs, int retryTimes) { return null; diff --git a/client/src/main/java/org/apache/uniffle/client/api/ShuffleWriteClient.java b/client/src/main/java/org/apache/uniffle/client/api/ShuffleWriteClient.java index 33980c92f1..717c0bcb65 100644 --- a/client/src/main/java/org/apache/uniffle/client/api/ShuffleWriteClient.java +++ b/client/src/main/java/org/apache/uniffle/client/api/ShuffleWriteClient.java @@ -37,16 +37,6 @@ public interface ShuffleWriteClient { - default SendShuffleDataResult sendShuffleData( - String appId, - int stageAttemptNumber, - List shuffleBlockInfoList, - Supplier needCancelRequest) { - throw new UnsupportedOperationException( - this.getClass().getName() - + " doesn't implement getShuffleAssignments with faultyServerIds"); - } - SendShuffleDataResult sendShuffleData( String appId, List shuffleBlockInfoList, @@ -72,7 +62,6 @@ default void registerShuffle( remoteStorage, dataDistributionType, maxConcurrencyPerPartitionToWrite, - 0, null, Collections.emptyMap()); } @@ -94,7 +83,6 @@ default void registerShuffle( remoteStorage, dataDistributionType, maxConcurrencyPerPartitionToWrite, - 0, null, properties); } @@ -107,7 +95,6 @@ default void registerShuffle( RemoteStorageInfo remoteStorage, ShuffleDataDistributionType dataDistributionType, int maxConcurrencyPerPartitionToWrite, - int stageAttemptNumber, MergeContext mergeContext) { registerShuffle( shuffleServerInfo, @@ -117,7 +104,6 @@ default void registerShuffle( remoteStorage, dataDistributionType, maxConcurrencyPerPartitionToWrite, - stageAttemptNumber, mergeContext, Collections.emptyMap()); } @@ -130,7 +116,6 @@ void registerShuffle( RemoteStorageInfo remoteStorage, ShuffleDataDistributionType dataDistributionType, int maxConcurrencyPerPartitionToWrite, - int stageAttemptNumber, MergeContext mergeContext, Map properties); @@ -155,15 +140,6 @@ void reportShuffleResult( long taskAttemptId, int bitmapNum); - default void reportShuffleResult( - Map>> serverToPartitionToBlockIds, - String appId, - int shuffleId, - long taskAttemptId, - int bitmapNum, - Set reportFailureServers, - boolean enableWriteFailureRetry) {} - ShuffleAssignmentsInfo getShuffleAssignments( String appId, int shuffleId, @@ -173,9 +149,6 @@ ShuffleAssignmentsInfo getShuffleAssignments( int assignmentShuffleServerNumber, int estimateTaskConcurrency, Set faultyServerIds, - int stageId, - int stageAttemptNumber, - boolean reassign, long retryIntervalMs, int retryTimes); @@ -197,9 +170,6 @@ default ShuffleAssignmentsInfo getShuffleAssignments( assignmentShuffleServerNumber, estimateTaskConcurrency, faultyServerIds, - -1, - 0, - false, 0, 0); } diff --git a/client/src/main/java/org/apache/uniffle/client/impl/ShuffleWriteClientImpl.java b/client/src/main/java/org/apache/uniffle/client/impl/ShuffleWriteClientImpl.java index 440237ee32..6c87e37da7 100644 --- a/client/src/main/java/org/apache/uniffle/client/impl/ShuffleWriteClientImpl.java +++ b/client/src/main/java/org/apache/uniffle/client/impl/ShuffleWriteClientImpl.java @@ -91,7 +91,6 @@ import org.apache.uniffle.common.config.RssConf; import org.apache.uniffle.common.exception.RssException; import org.apache.uniffle.common.exception.RssFetchFailedException; -import org.apache.uniffle.common.exception.RssSendFailedException; import org.apache.uniffle.common.rpc.StatusCode; import org.apache.uniffle.common.util.BlockIdLayout; import org.apache.uniffle.common.util.JavaUtils; @@ -166,7 +165,6 @@ public ShuffleWriteClientImpl(ShuffleClientFactory.WriteClientBuilder builder) { private boolean sendShuffleDataAsync( String appId, - int stageAttemptNumber, Map>>> serverToBlocks, Map> serverToBlockIds, Map blockIdsSendSuccessTracker, @@ -197,11 +195,7 @@ private boolean sendShuffleDataAsync( // todo: compact unnecessary blocks that reach replicaWrite RssSendShuffleDataRequest request = new RssSendShuffleDataRequest( - appId, - stageAttemptNumber, - retryMax, - retryIntervalMax, - shuffleIdToBlocks); + appId, retryMax, retryIntervalMax, shuffleIdToBlocks); long s = System.currentTimeMillis(); RssSendShuffleDataResponse response = getShuffleServerClient(ssi).sendShuffleData(request); @@ -343,20 +337,10 @@ void genServerToBlocks( }); } - @Override - @VisibleForTesting - public SendShuffleDataResult sendShuffleData( - String appId, - List shuffleBlockInfoList, - Supplier needCancelRequest) { - return sendShuffleData(appId, 0, shuffleBlockInfoList, needCancelRequest); - } - /** The batch of sending belongs to the same task */ @Override public SendShuffleDataResult sendShuffleData( String appId, - int stageAttemptNumber, List shuffleBlockInfoList, Supplier needCancelRequest) { @@ -440,7 +424,6 @@ public SendShuffleDataResult sendShuffleData( boolean isAllSuccess = sendShuffleDataAsync( appId, - stageAttemptNumber, primaryServerToBlocks, primaryServerToBlockIds, blockIdsSendSuccessTracker, @@ -458,7 +441,6 @@ public SendShuffleDataResult sendShuffleData( LOG.info("The sending of primary round is failed partially, so start the secondary round"); sendShuffleDataAsync( appId, - stageAttemptNumber, secondaryServerToBlocks, secondaryServerToBlockIds, blockIdsSendSuccessTracker, @@ -591,7 +573,6 @@ public void registerShuffle( RemoteStorageInfo remoteStorage, ShuffleDataDistributionType dataDistributionType, int maxConcurrencyPerPartitionToWrite, - int stageAttemptNumber, MergeContext mergeContext, Map properties) { String user = null; @@ -610,7 +591,6 @@ public void registerShuffle( user, dataDistributionType, maxConcurrencyPerPartitionToWrite, - stageAttemptNumber, mergeContext, properties); RssRegisterShuffleResponse response = @@ -676,9 +656,6 @@ public ShuffleAssignmentsInfo getShuffleAssignments( int assignmentShuffleServerNumber, int estimateTaskConcurrency, Set faultyServerIds, - int stageId, - int stageAttemptNumber, - boolean reassign, long retryIntervalMs, int retryTimes) { RssGetShuffleAssignmentsRequest request = @@ -692,9 +669,6 @@ public ShuffleAssignmentsInfo getShuffleAssignments( assignmentShuffleServerNumber, estimateTaskConcurrency, faultyServerIds, - stageId, - stageAttemptNumber, - reassign, retryIntervalMs, retryTimes); @@ -732,25 +706,6 @@ public void reportShuffleResult( int shuffleId, long taskAttemptId, int bitmapNum) { - reportShuffleResult( - serverToPartitionToBlockIds, - appId, - shuffleId, - taskAttemptId, - bitmapNum, - Sets.newConcurrentHashSet(), - false); - } - - @Override - public void reportShuffleResult( - Map>> serverToPartitionToBlockIds, - String appId, - int shuffleId, - long taskAttemptId, - int bitmapNum, - Set reportFailureServers, - boolean enableWriteFailureRetry) { // record blockId count for quora check,but this is not a good realization. Map blockReportTracker = createBlockReportTracker(serverToPartitionToBlockIds); for (Map.Entry>> entry : @@ -788,13 +743,6 @@ public void reportShuffleResult( response.getStatusCode(), System.currentTimeMillis() - start); recordFailedBlockIds(blockReportTracker, requestBlockIds); - if (enableWriteFailureRetry) { - // The failed Shuffle Server is recorded and corresponding exceptions are raised only - // when the retry function is started. - reportFailureServers.add(ssi); - throw new RssSendFailedException( - "Throw an exception because the report shuffle result status code is not SUCCESS."); - } } } catch (Exception e) { LOG.warn( @@ -806,13 +754,6 @@ public void reportShuffleResult( + shuffleId + "]"); recordFailedBlockIds(blockReportTracker, requestBlockIds); - if (enableWriteFailureRetry) { - // The failed Shuffle Server is recorded and corresponding exceptions are raised only when - // the retry function is started. - reportFailureServers.add(ssi); - throw new RssSendFailedException( - "Throw an exception because the report shuffle result status code is not SUCCESS."); - } } } if (blockReportTracker.values().stream().anyMatch(cnt -> cnt < replicaWrite)) { diff --git a/client/src/test/java/org/apache/uniffle/client/record/reader/MockedShuffleWriteClient.java b/client/src/test/java/org/apache/uniffle/client/record/reader/MockedShuffleWriteClient.java index d856292f20..246f958298 100644 --- a/client/src/test/java/org/apache/uniffle/client/record/reader/MockedShuffleWriteClient.java +++ b/client/src/test/java/org/apache/uniffle/client/record/reader/MockedShuffleWriteClient.java @@ -63,7 +63,6 @@ public void registerShuffle( RemoteStorageInfo remoteStorage, ShuffleDataDistributionType dataDistributionType, int maxConcurrencyPerPartitionToWrite, - int stageAttemptNumber, RssProtos.MergeContext mergeContext, Map properties) {} @@ -120,9 +119,6 @@ public ShuffleAssignmentsInfo getShuffleAssignments( int assignmentShuffleServerNumber, int estimateTaskConcurrency, Set faultyServerIds, - int stageId, - int stageAttemptNumber, - boolean reassign, long retryIntervalMs, int retryTimes) { return null; diff --git a/common/src/main/java/org/apache/uniffle/common/netty/protocol/SendShuffleDataRequest.java b/common/src/main/java/org/apache/uniffle/common/netty/protocol/SendShuffleDataRequest.java index cde9a3ee26..c3e3375c82 100644 --- a/common/src/main/java/org/apache/uniffle/common/netty/protocol/SendShuffleDataRequest.java +++ b/common/src/main/java/org/apache/uniffle/common/netty/protocol/SendShuffleDataRequest.java @@ -30,8 +30,6 @@ public class SendShuffleDataRequest extends RequestMessage { private String appId; private int shuffleId; - - private int stageAttemptNumber; private long requireId; private Map> partitionToBlocks; private long timestamp; @@ -43,24 +41,12 @@ public SendShuffleDataRequest( long requireId, Map> partitionToBlocks, long timestamp) { - this(requestId, appId, shuffleId, 0, requireId, partitionToBlocks, timestamp); - } - - public SendShuffleDataRequest( - long requestId, - String appId, - int shuffleId, - int stageAttemptNumber, - long requireId, - Map> partitionToBlocks, - long timestamp) { super(requestId); this.appId = appId; this.shuffleId = shuffleId; this.requireId = requireId; this.partitionToBlocks = partitionToBlocks; this.timestamp = timestamp; - this.stageAttemptNumber = stageAttemptNumber; } @Override @@ -174,10 +160,6 @@ public void setTimestamp(long timestamp) { this.timestamp = timestamp; } - public int getStageAttemptNumber() { - return stageAttemptNumber; - } - @Override public String getOperationType() { return "sendShuffleData"; diff --git a/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorGrpcService.java b/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorGrpcService.java index 489199f502..4bed0a9905 100644 --- a/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorGrpcService.java +++ b/coordinator/src/main/java/org/apache/uniffle/coordinator/CoordinatorGrpcService.java @@ -156,22 +156,19 @@ public void getShuffleAssignments( auditContext.withArgs( String.format( "shuffleId=%d, partitionNum=%d, partitionNumPerRange=%d, replica=%d, requiredTags=%s, " - + "requiredShuffleServerNumber=%d, faultyServerIds=%s, stageId=%d, stageAttemptNumber=%d, isReassign=%b", + + "requiredShuffleServerNumber=%d, faultyServerIds=%s", shuffleId, partitionNum, partitionNumPerRange, replica, requiredTags, requiredShuffleServerNumber, - faultyServerIds, - request.getStageId(), - request.getStageAttemptNumber(), - request.getReassign())); + faultyServerIds)); LOG.info( "Request of getShuffleAssignments for appId[{}], shuffleId[{}], partitionNum[{}]," + " partitionNumPerRange[{}], replica[{}], requiredTags[{}], requiredShuffleServerNumber[{}]," - + " faultyServerIds[{}], stageId[{}], stageAttemptNumber[{}], isReassign[{}]", + + " faultyServerIds[{}]", appId, shuffleId, partitionNum, @@ -179,10 +176,7 @@ public void getShuffleAssignments( replica, requiredTags, requiredShuffleServerNumber, - faultyServerIds.size(), - request.getStageId(), - request.getStageAttemptNumber(), - request.getReassign()); + faultyServerIds.size()); GetShuffleAssignmentsResponse response = null; try { diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/RemoteMergeShuffleWithRssClientTest.java b/integration-test/common/src/test/java/org/apache/uniffle/test/RemoteMergeShuffleWithRssClientTest.java index e32cdedd0a..9d2a9d187b 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/RemoteMergeShuffleWithRssClientTest.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/RemoteMergeShuffleWithRssClientTest.java @@ -164,7 +164,6 @@ public void remoteMergeWriteReadTest(String classes) throws Exception { new RemoteStorageInfo(""), ShuffleDataDistributionType.NORMAL, 0, - 0, RssProtos.MergeContext.newBuilder() .setKeyClass(keyClass.getName()) .setValueClass(valueClass.getName()) @@ -339,7 +338,6 @@ public void remoteMergeWriteReadTestWithCombine(String classes) throws Exception new RemoteStorageInfo(""), ShuffleDataDistributionType.NORMAL, 0, - 0, RssProtos.MergeContext.newBuilder() .setKeyClass(keyClass.getName()) .setValueClass(valueClass.getName()) @@ -522,7 +520,6 @@ public void remoteMergeWriteReadTestMultiPartition(String classes) throws Except new RemoteStorageInfo(""), ShuffleDataDistributionType.NORMAL, 0, - 0, RssProtos.MergeContext.newBuilder() .setKeyClass(keyClass.getName()) .setValueClass(valueClass.getName()) @@ -740,7 +737,6 @@ public void remoteMergeWriteReadTestMultiPartitionWithCombine(String classes) th new RemoteStorageInfo(""), ShuffleDataDistributionType.NORMAL, 0, - 0, RssProtos.MergeContext.newBuilder() .setKeyClass(keyClass.getName()) .setValueClass(valueClass.getName()) diff --git a/integration-test/common/src/test/java/org/apache/uniffle/test/RemoteMergeShuffleWithRssClientTestWhenShuffleFlushed.java b/integration-test/common/src/test/java/org/apache/uniffle/test/RemoteMergeShuffleWithRssClientTestWhenShuffleFlushed.java index 0e83ed0cd8..917820b050 100644 --- a/integration-test/common/src/test/java/org/apache/uniffle/test/RemoteMergeShuffleWithRssClientTestWhenShuffleFlushed.java +++ b/integration-test/common/src/test/java/org/apache/uniffle/test/RemoteMergeShuffleWithRssClientTestWhenShuffleFlushed.java @@ -178,7 +178,6 @@ public void remoteMergeWriteReadTest(String classes) throws Exception { new RemoteStorageInfo(""), ShuffleDataDistributionType.NORMAL, -1, - 0, RssProtos.MergeContext.newBuilder() .setKeyClass(keyClass.getName()) .setValueClass(valueClass.getName()) @@ -353,7 +352,6 @@ public void remoteMergeWriteReadTestWithCombine(String classes) throws Exception new RemoteStorageInfo(""), ShuffleDataDistributionType.NORMAL, -1, - 0, RssProtos.MergeContext.newBuilder() .setKeyClass(keyClass.getName()) .setValueClass(valueClass.getName()) @@ -537,7 +535,6 @@ public void remoteMergeWriteReadTestMultiPartition(String classes) throws Except new RemoteStorageInfo(""), ShuffleDataDistributionType.NORMAL, -1, - 0, RssProtos.MergeContext.newBuilder() .setKeyClass(keyClass.getName()) .setValueClass(valueClass.getName()) @@ -756,7 +753,6 @@ public void remoteMergeWriteReadTestMultiPartitionWithCombine(String classes) th new RemoteStorageInfo(""), ShuffleDataDistributionType.NORMAL, -1, - 0, RssProtos.MergeContext.newBuilder() .setKeyClass(keyClass.getName()) .setValueClass(valueClass.getName()) diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RSSStageDynamicServerReWriteTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RSSStageDynamicServerReWriteTest.java deleted file mode 100644 index 26f43542db..0000000000 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RSSStageDynamicServerReWriteTest.java +++ /dev/null @@ -1,105 +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.test; - -import java.io.File; -import java.util.List; -import java.util.Map; - -import com.google.common.collect.Maps; -import org.apache.spark.SparkConf; -import org.apache.spark.shuffle.RssSparkConfig; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -import org.apache.uniffle.common.rpc.ServerType; -import org.apache.uniffle.coordinator.CoordinatorConf; -import org.apache.uniffle.server.MockedGrpcServer; -import org.apache.uniffle.server.ShuffleServerConf; -import org.apache.uniffle.storage.util.StorageType; - -public class RSSStageDynamicServerReWriteTest extends SparkTaskFailureIntegrationTestBase { - @BeforeAll - public static void setupServers(@TempDir File tmpDir) throws Exception { - CoordinatorConf coordinatorConf = coordinatorConfWithoutPort(); - Map dynamicConf = Maps.newHashMap(); - dynamicConf.put(CoordinatorConf.COORDINATOR_REMOTE_STORAGE_PATH.key(), HDFS_URI + "rss/test"); - dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.MEMORY_LOCALFILE.name()); - addDynamicConf(coordinatorConf, dynamicConf); - storeCoordinatorConf(coordinatorConf); - prepareServerConf(0, tmpDir, true, ServerType.GRPC); - prepareServerConf(1, tmpDir, false, ServerType.GRPC); - prepareServerConf(2, tmpDir, false, ServerType.GRPC); - prepareServerConf(3, tmpDir, true, ServerType.GRPC_NETTY); - prepareServerConf(4, tmpDir, false, ServerType.GRPC_NETTY); - prepareServerConf(5, tmpDir, false, ServerType.GRPC_NETTY); - startServersWithRandomPorts(); - - // Set the sending block data timeout for the first shuffleServer - ((MockedGrpcServer) grpcShuffleServers.get(2).getServer()) - .getService() - .enableMockSendDataFailed(true); - - ((MockedGrpcServer) nettyShuffleServers.get(2).getServer()) - .getService() - .enableMockSendDataFailed(true); - } - - public static void prepareServerConf( - int id, File tmpDir, boolean abnormalFlag, ServerType serverType) { - ShuffleServerConf shuffleServerConf = shuffleServerConfWithoutPort(id, tmpDir, serverType); - shuffleServerConf.setLong("rss.server.app.expired.withoutHeartbeat", 8000); - shuffleServerConf.setLong("rss.server.heartbeat.interval", 5000); - shuffleServerConf.setString("rss.storage.type", StorageType.MEMORY_LOCALFILE.name()); - shuffleServerConf.set(ShuffleServerConf.SERVER_RPC_AUDIT_LOG_ENABLED, false); - - if (abnormalFlag) { - storeMockShuffleServerConf(shuffleServerConf); - } else { - storeShuffleServerConf(shuffleServerConf); - } - } - - @Override - public Map runTest(SparkSession spark, String fileName) throws Exception { - List rows = - spark.range(0, 1000, 1, 4).repartition(2).groupBy("id").count().collectAsList(); - Map result = Maps.newHashMap(); - for (Row row : rows) { - result.put(row.get(0).toString(), row.getLong(1)); - } - return result; - } - - @Override - public void updateSparkConfCustomer(SparkConf sparkConf) { - super.updateSparkConfCustomer(sparkConf); - sparkConf.set( - RssSparkConfig.SPARK_RSS_CONFIG_PREFIX - + RssSparkConfig.RSS_RESUBMIT_STAGE_WITH_WRITE_FAILURE_ENABLED.key(), - "true"); - } - - @Test - public void testRSSStageResubmit() throws Exception { - run(); - } -} diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RSSStageResubmitTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RSSStageResubmitTest.java index b9453ff9af..8cfc85efec 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RSSStageResubmitTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/RSSStageResubmitTest.java @@ -79,9 +79,7 @@ public Map runTest(SparkSession spark, String fileName) throws Exception { public void updateSparkConfCustomer(SparkConf sparkConf) { super.updateSparkConfCustomer(sparkConf); sparkConf.set( - RssSparkConfig.SPARK_RSS_CONFIG_PREFIX - + RssSparkConfig.RSS_RESUBMIT_STAGE_WITH_FETCH_FAILURE_ENABLED, - "true"); + RssSparkConfig.SPARK_RSS_CONFIG_PREFIX + RssSparkConfig.RSS_RESUBMIT_STAGE_ENABLED, "true"); } @Test diff --git a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SimpleShuffleServerManagerTest.java b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SimpleShuffleServerManagerTest.java index 9033cc827a..f8ff83eb27 100644 --- a/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SimpleShuffleServerManagerTest.java +++ b/integration-test/spark-common/src/test/java/org/apache/uniffle/test/SimpleShuffleServerManagerTest.java @@ -33,12 +33,12 @@ public class SimpleShuffleServerManagerTest extends ShuffleServerManagerTestBase public void testClientAndServerConnections() { RssShuffleManagerInterface dummy = new DummyRssShuffleManager(); RssReportShuffleFetchFailureRequest req = - new RssReportShuffleFetchFailureRequest(dummy.getAppId(), 0, 0, 0, null); + new RssReportShuffleFetchFailureRequest(dummy.getAppId(), 0, 0, 0, 0, 0, null); RssReportShuffleFetchFailureResponse res = client.reportShuffleFetchFailure(req); assertEquals(StatusCode.SUCCESS, res.getStatusCode()); // wrong appId - req = new RssReportShuffleFetchFailureRequest("wrongAppId", 0, 0, 0, null); + req = new RssReportShuffleFetchFailureRequest("wrongAppId", 0, 0, 0, 0, 0, null); res = client.reportShuffleFetchFailure(req); assertEquals(StatusCode.INVALID_REQUEST, res.getStatusCode()); } diff --git a/integration-test/spark3/src/test/java/org/apache/uniffle/test/ReassignAndStageRetryTest.java b/integration-test/spark3/src/test/java/org/apache/uniffle/test/ReassignAndStageRetryTest.java deleted file mode 100644 index 663a176c87..0000000000 --- a/integration-test/spark3/src/test/java/org/apache/uniffle/test/ReassignAndStageRetryTest.java +++ /dev/null @@ -1,71 +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.test; - -import org.apache.spark.SparkConf; - -import org.apache.uniffle.server.MockedGrpcServer; -import org.apache.uniffle.server.ShuffleServer; -import org.apache.uniffle.server.buffer.ShuffleBufferManager; - -import static org.apache.spark.shuffle.RssSparkConfig.RSS_PARTITION_REASSIGN_BLOCK_RETRY_MAX_TIMES; -import static org.apache.uniffle.client.util.RssClientConfig.RSS_CLIENT_ASSIGNMENT_SHUFFLE_SERVER_NUMBER; -import static org.apache.uniffle.client.util.RssClientConfig.RSS_CLIENT_RETRY_MAX; -import static org.apache.uniffle.client.util.RssClientConfig.RSS_RESUBMIT_STAGE; -import static org.apache.uniffle.common.config.RssClientConf.RSS_CLIENT_REASSIGN_ENABLED; - -/** - * This class is to test the compatibility of reassign and stage retry mechanism that were enabled - * at the same time. - */ -public class ReassignAndStageRetryTest extends PartitionBlockDataReassignMultiTimesTest { - - @Override - public void updateSparkConfCustomer(SparkConf sparkConf) { - sparkConf.set("spark.task.maxFailures", String.valueOf(1)); - sparkConf.set("spark." + RSS_RESUBMIT_STAGE, "true"); - - sparkConf.set("spark.sql.shuffle.partitions", "4"); - sparkConf.set("spark." + RSS_CLIENT_RETRY_MAX, "2"); - sparkConf.set("spark." + RSS_CLIENT_ASSIGNMENT_SHUFFLE_SERVER_NUMBER, "1"); - sparkConf.set("spark." + RSS_CLIENT_REASSIGN_ENABLED.key(), "true"); - sparkConf.set("spark." + RSS_PARTITION_REASSIGN_BLOCK_RETRY_MAX_TIMES.key(), "1"); - - // simulate the grpc servers has different free memory - // and make the assign priority seq: g1 -> g2 -> g3 - ShuffleServer g1 = grpcShuffleServers.get(0); - ShuffleBufferManager bufferManager = g1.getShuffleBufferManager(); - bufferManager.setUsedMemory(bufferManager.getCapacity() - 3000000); - g1.sendHeartbeat(); - - ShuffleServer g2 = grpcShuffleServers.get(1); - bufferManager = g2.getShuffleBufferManager(); - bufferManager.setUsedMemory(bufferManager.getCapacity() - 2000000); - g2.sendHeartbeat(); - - ShuffleServer g3 = grpcShuffleServers.get(2); - bufferManager = g3.getShuffleBufferManager(); - bufferManager.setUsedMemory(bufferManager.getCapacity() - 1000000); - g3.sendHeartbeat(); - - // This will make the partition of g1 reassign to g2 servers. - ((MockedGrpcServer) g1.getServer()).getService().setMockSendDataFailedStageNumber(0); - // And then reassign to g3. But reassign max times reaches due to max reassign times. - ((MockedGrpcServer) g2.getServer()).getService().setMockSendDataFailedStageNumber(0); - } -} diff --git a/internal-client/src/main/java/org/apache/uniffle/client/api/ShuffleManagerClient.java b/internal-client/src/main/java/org/apache/uniffle/client/api/ShuffleManagerClient.java index 381ad706f3..62c37d28bf 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/api/ShuffleManagerClient.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/api/ShuffleManagerClient.java @@ -17,6 +17,7 @@ package org.apache.uniffle.client.api; +import org.apache.uniffle.client.request.RssAppUniffleShuffleIdRequest; import org.apache.uniffle.client.request.RssGetShuffleResultForMultiPartRequest; import org.apache.uniffle.client.request.RssGetShuffleResultRequest; import org.apache.uniffle.client.request.RssPartitionToShuffleServerRequest; @@ -24,15 +25,13 @@ import org.apache.uniffle.client.request.RssReportShuffleFetchFailureRequest; import org.apache.uniffle.client.request.RssReportShuffleReadMetricRequest; import org.apache.uniffle.client.request.RssReportShuffleResultRequest; -import org.apache.uniffle.client.request.RssReportShuffleWriteFailureRequest; import org.apache.uniffle.client.request.RssReportShuffleWriteMetricRequest; +import org.apache.uniffle.client.response.RssAppUniffleShuffleIdResponse; import org.apache.uniffle.client.response.RssGetShuffleResultResponse; import org.apache.uniffle.client.response.RssReassignOnBlockSendFailureResponse; -import org.apache.uniffle.client.response.RssReassignOnStageRetryResponse; import org.apache.uniffle.client.response.RssReportShuffleFetchFailureResponse; import org.apache.uniffle.client.response.RssReportShuffleReadMetricResponse; import org.apache.uniffle.client.response.RssReportShuffleResultResponse; -import org.apache.uniffle.client.response.RssReportShuffleWriteFailureResponse; import org.apache.uniffle.client.response.RssReportShuffleWriteMetricResponse; import org.apache.uniffle.common.util.StatefulCloseable; @@ -40,29 +39,11 @@ public interface ShuffleManagerClient extends StatefulCloseable { RssReportShuffleFetchFailureResponse reportShuffleFetchFailure( RssReportShuffleFetchFailureRequest request); - /** - * In Stage Retry mode,Gets the mapping between partitions and ShuffleServer from the - * ShuffleManager server. - * - * @param req request - * @return RssPartitionToShuffleServerResponse - */ - RssReassignOnStageRetryResponse getPartitionToShufflerServerWithStageRetry( - RssPartitionToShuffleServerRequest req); + RssAppUniffleShuffleIdResponse getUniffleShuffleId(RssAppUniffleShuffleIdRequest req); - /** - * In Block Retry mode,Gets the mapping between partitions and ShuffleServer from the - * ShuffleManager server. - * - * @param req request - * @return RssPartitionToShuffleServerResponse - */ - RssReassignOnBlockSendFailureResponse getPartitionToShufflerServerWithBlockRetry( + RssReassignOnBlockSendFailureResponse getPartitionToShufflerServer( RssPartitionToShuffleServerRequest req); - RssReportShuffleWriteFailureResponse reportShuffleWriteFailure( - RssReportShuffleWriteFailureRequest req); - RssReassignOnBlockSendFailureResponse reassignOnBlockSendFailure( RssReassignOnBlockSendFailureRequest request); diff --git a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/CoordinatorGrpcClient.java b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/CoordinatorGrpcClient.java index 7f35b5b591..5ff354dd2b 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/CoordinatorGrpcClient.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/CoordinatorGrpcClient.java @@ -174,10 +174,7 @@ public RssProtos.GetShuffleAssignmentsResponse doGetShuffleAssignments( Set requiredTags, int assignmentShuffleServerNumber, int estimateTaskConcurrency, - Set faultyServerIds, - int stageId, - int stageAttemptNumber, - boolean reassign) { + Set faultyServerIds) { RssProtos.GetShuffleServerRequest getServerRequest = RssProtos.GetShuffleServerRequest.newBuilder() .setApplicationId(appId) @@ -189,9 +186,6 @@ public RssProtos.GetShuffleAssignmentsResponse doGetShuffleAssignments( .setAssignmentShuffleServerNumber(assignmentShuffleServerNumber) .setEstimateTaskConcurrency(estimateTaskConcurrency) .addAllFaultyServerIds(faultyServerIds) - .setStageId(stageId) - .setStageAttemptNumber(stageAttemptNumber) - .setReassign(reassign) .build(); return blockingStub.getShuffleAssignments(getServerRequest); @@ -275,10 +269,7 @@ public RssGetShuffleAssignmentsResponse getShuffleAssignments( request.getRequiredTags(), request.getAssignmentShuffleServerNumber(), request.getEstimateTaskConcurrency(), - request.getFaultyServerIds(), - request.getStageId(), - request.getStageAttemptNumber(), - request.isReassign()); + request.getFaultyServerIds()); RssGetShuffleAssignmentsResponse response; RssProtos.StatusCode statusCode = rpcResponse.getStatus(); diff --git a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleManagerGrpcClient.java b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleManagerGrpcClient.java index 0040e56ef8..f16c0feac1 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleManagerGrpcClient.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleManagerGrpcClient.java @@ -23,6 +23,7 @@ import org.slf4j.LoggerFactory; import org.apache.uniffle.client.api.ShuffleManagerClient; +import org.apache.uniffle.client.request.RssAppUniffleShuffleIdRequest; import org.apache.uniffle.client.request.RssGetShuffleResultForMultiPartRequest; import org.apache.uniffle.client.request.RssGetShuffleResultRequest; import org.apache.uniffle.client.request.RssPartitionToShuffleServerRequest; @@ -30,15 +31,13 @@ import org.apache.uniffle.client.request.RssReportShuffleFetchFailureRequest; import org.apache.uniffle.client.request.RssReportShuffleReadMetricRequest; import org.apache.uniffle.client.request.RssReportShuffleResultRequest; -import org.apache.uniffle.client.request.RssReportShuffleWriteFailureRequest; import org.apache.uniffle.client.request.RssReportShuffleWriteMetricRequest; +import org.apache.uniffle.client.response.RssAppUniffleShuffleIdResponse; import org.apache.uniffle.client.response.RssGetShuffleResultResponse; import org.apache.uniffle.client.response.RssReassignOnBlockSendFailureResponse; -import org.apache.uniffle.client.response.RssReassignOnStageRetryResponse; import org.apache.uniffle.client.response.RssReportShuffleFetchFailureResponse; import org.apache.uniffle.client.response.RssReportShuffleReadMetricResponse; import org.apache.uniffle.client.response.RssReportShuffleResultResponse; -import org.apache.uniffle.client.response.RssReportShuffleWriteFailureResponse; import org.apache.uniffle.client.response.RssReportShuffleWriteMetricResponse; import org.apache.uniffle.common.exception.RssException; import org.apache.uniffle.proto.RssProtos; @@ -91,40 +90,22 @@ public RssReportShuffleFetchFailureResponse reportShuffleFetchFailure( } @Override - public RssReassignOnStageRetryResponse getPartitionToShufflerServerWithStageRetry( - RssPartitionToShuffleServerRequest req) { - RssProtos.PartitionToShuffleServerRequest protoRequest = req.toProto(); - RssProtos.ReassignOnStageRetryResponse partitionToShufflerServer = - getBlockingStub().getPartitionToShufflerServerWithStageRetry(protoRequest); - RssReassignOnStageRetryResponse rssReassignOnStageRetryResponse = - RssReassignOnStageRetryResponse.fromProto(partitionToShufflerServer); - return rssReassignOnStageRetryResponse; - } - - @Override - public RssReassignOnBlockSendFailureResponse getPartitionToShufflerServerWithBlockRetry( + public RssReassignOnBlockSendFailureResponse getPartitionToShufflerServer( RssPartitionToShuffleServerRequest req) { RssProtos.PartitionToShuffleServerRequest protoRequest = req.toProto(); RssProtos.ReassignOnBlockSendFailureResponse partitionToShufflerServer = - getBlockingStub().getPartitionToShufflerServerWithBlockRetry(protoRequest); + getBlockingStub().getPartitionToShufflerServer(protoRequest); RssReassignOnBlockSendFailureResponse rssReassignOnBlockSendFailureResponse = RssReassignOnBlockSendFailureResponse.fromProto(partitionToShufflerServer); return rssReassignOnBlockSendFailureResponse; } @Override - public RssReportShuffleWriteFailureResponse reportShuffleWriteFailure( - RssReportShuffleWriteFailureRequest request) { - RssProtos.ReportShuffleWriteFailureRequest protoRequest = request.toProto(); - try { - RssProtos.ReportShuffleWriteFailureResponse response = - getBlockingStub().reportShuffleWriteFailure(protoRequest); - return RssReportShuffleWriteFailureResponse.fromProto(response); - } catch (Exception e) { - String msg = "Report shuffle fetch failure to host:port[" + host + ":" + port + "] failed"; - LOG.warn(msg, e); - throw new RssException(msg, e); - } + public RssAppUniffleShuffleIdResponse getUniffleShuffleId(RssAppUniffleShuffleIdRequest req) { + RssProtos.AppUniffleShuffleIdRequest appUniffleShuffleIdRequest = req.toProto(); + RssProtos.AppUniffleShuffleIdResponse appUniffleShuffleIdResponse = + getBlockingStub().getUniffleShuffleId(appUniffleShuffleIdRequest); + return RssAppUniffleShuffleIdResponse.fromProto(appUniffleShuffleIdResponse); } @Override 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 82dd7c9b3c..9512450817 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 @@ -217,7 +217,6 @@ private ShuffleRegisterResponse doRegisterShuffle( String user, ShuffleDataDistributionType dataDistributionType, int maxConcurrencyPerPartitionToWrite, - int stageAttemptNumber, MergeContext mergeContext, Map properties) { ShuffleRegisterRequest.Builder reqBuilder = ShuffleRegisterRequest.newBuilder(); @@ -228,7 +227,6 @@ private ShuffleRegisterResponse doRegisterShuffle( .setShuffleDataDistribution(RssProtos.DataDistribution.valueOf(dataDistributionType.name())) .setMaxConcurrencyPerPartitionToWrite(maxConcurrencyPerPartitionToWrite) .addAllPartitionRanges(toShufflePartitionRanges(partitionRanges)) - .setStageAttemptNumber(stageAttemptNumber) .putAllProperties(properties); if (mergeContext != null) { reqBuilder.setMergeContext(mergeContext); @@ -508,7 +506,6 @@ public RssRegisterShuffleResponse registerShuffle(RssRegisterShuffleRequest requ request.getUser(), request.getDataDistributionType(), request.getMaxConcurrencyPerPartitionToWrite(), - request.getStageAttemptNumber(), request.getMergeContext(), request.getProperties()); @@ -541,7 +538,6 @@ public RssSendShuffleDataResponse sendShuffleData(RssSendShuffleDataRequest requ String appId = request.getAppId(); Map>> shuffleIdToBlocks = request.getShuffleIdToBlocks(); - int stageAttemptNumber = request.getStageAttemptNumber(); boolean isSuccessful = true; AtomicReference failedStatusCode = new AtomicReference<>(StatusCode.INTERNAL_ERROR); @@ -614,7 +610,6 @@ public RssSendShuffleDataResponse sendShuffleData(RssSendShuffleDataRequest requ .setRequireBufferId(requireId) .addAllShuffleData(shuffleData) .setTimestamp(start) - .setStageAttemptNumber(stageAttemptNumber) .build(); SendShuffleDataResponse response = getBlockingStub().sendShuffleData(rpcRequest); if (LOG.isDebugEnabled()) { diff --git a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerGrpcNettyClient.java b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerGrpcNettyClient.java index 07e91ea9e9..74477b4ec3 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerGrpcNettyClient.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/impl/grpc/ShuffleServerGrpcNettyClient.java @@ -136,7 +136,6 @@ public ClientInfo getClientInfo() { public RssSendShuffleDataResponse sendShuffleData(RssSendShuffleDataRequest request) { Map>> shuffleIdToBlocks = request.getShuffleIdToBlocks(); - int stageAttemptNumber = request.getStageAttemptNumber(); boolean isSuccessful = true; AtomicReference failedStatusCode = new AtomicReference<>(StatusCode.INTERNAL_ERROR); Set needSplitPartitionIds = new HashSet<>(); @@ -163,7 +162,6 @@ public RssSendShuffleDataResponse sendShuffleData(RssSendShuffleDataRequest requ requestId(), request.getAppId(), shuffleId, - stageAttemptNumber, 0L, stb.getValue(), System.currentTimeMillis()); diff --git a/internal-client/src/main/java/org/apache/uniffle/client/request/RssAppUniffleShuffleIdRequest.java b/internal-client/src/main/java/org/apache/uniffle/client/request/RssAppUniffleShuffleIdRequest.java new file mode 100644 index 0000000000..9543cfc078 --- /dev/null +++ b/internal-client/src/main/java/org/apache/uniffle/client/request/RssAppUniffleShuffleIdRequest.java @@ -0,0 +1,50 @@ +/* + * 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.client.request; + +import org.apache.uniffle.proto.RssProtos; + +public class RssAppUniffleShuffleIdRequest { + private final int shuffleId; + private final String appShuffleIdentifier; + private final boolean isWriter; + + public RssAppUniffleShuffleIdRequest( + int shuffleId, String appShuffleIdentifier, boolean isWriter) { + this.shuffleId = shuffleId; + this.appShuffleIdentifier = appShuffleIdentifier; + this.isWriter = isWriter; + } + + public int getShuffleId() { + return shuffleId; + } + + public boolean isWriter() { + return isWriter; + } + + public RssProtos.AppUniffleShuffleIdRequest toProto() { + RssProtos.AppUniffleShuffleIdRequest.Builder builder = + RssProtos.AppUniffleShuffleIdRequest.newBuilder() + .setShuffleId(shuffleId) + .setAppShuffleIdentifier(appShuffleIdentifier) + .setIsWriter(isWriter); + return builder.build(); + } +} diff --git a/internal-client/src/main/java/org/apache/uniffle/client/request/RssGetShuffleAssignmentsRequest.java b/internal-client/src/main/java/org/apache/uniffle/client/request/RssGetShuffleAssignmentsRequest.java index 84e36e1038..0aa3e3347c 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/request/RssGetShuffleAssignmentsRequest.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/request/RssGetShuffleAssignmentsRequest.java @@ -33,9 +33,6 @@ public class RssGetShuffleAssignmentsRequest { private int assignmentShuffleServerNumber; private int estimateTaskConcurrency; private Set faultyServerIds; - private int stageId = -1; - private int stageAttemptNumber = 0; - private boolean reassign = false; private long retryIntervalMs; private int retryTimes; @@ -79,9 +76,6 @@ public RssGetShuffleAssignmentsRequest( assignmentShuffleServerNumber, estimateTaskConcurrency, faultyServerIds, - -1, - 0, - false, 0, 0); } @@ -96,9 +90,6 @@ public RssGetShuffleAssignmentsRequest( int assignmentShuffleServerNumber, int estimateTaskConcurrency, Set faultyServerIds, - int stageId, - int stageAttemptNumber, - boolean reassign, long retryIntervalMs, int retryTimes) { this.appId = appId; @@ -110,9 +101,6 @@ public RssGetShuffleAssignmentsRequest( this.assignmentShuffleServerNumber = assignmentShuffleServerNumber; this.estimateTaskConcurrency = estimateTaskConcurrency; this.faultyServerIds = faultyServerIds; - this.stageId = stageId; - this.stageAttemptNumber = stageAttemptNumber; - this.reassign = reassign; this.retryIntervalMs = retryIntervalMs; this.retryTimes = retryTimes; } @@ -153,18 +141,6 @@ public Set getFaultyServerIds() { return faultyServerIds; } - public int getStageId() { - return stageId; - } - - public int getStageAttemptNumber() { - return stageAttemptNumber; - } - - public boolean isReassign() { - return reassign; - } - public long getRetryIntervalMs() { return retryIntervalMs; } diff --git a/internal-client/src/main/java/org/apache/uniffle/client/request/RssPartitionToShuffleServerRequest.java b/internal-client/src/main/java/org/apache/uniffle/client/request/RssPartitionToShuffleServerRequest.java index 9e42d2c6a7..ba4d4a104a 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/request/RssPartitionToShuffleServerRequest.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/request/RssPartitionToShuffleServerRequest.java @@ -20,34 +20,21 @@ import org.apache.uniffle.proto.RssProtos; public class RssPartitionToShuffleServerRequest { - private int stageAttemptId; - private int stageAttemptNumber; private int shuffleId; - private boolean isWritePhase; - public RssPartitionToShuffleServerRequest( - int stageAttemptId, int stageAttemptNumber, int shuffleId, boolean isWritePhase) { - this.stageAttemptId = stageAttemptId; - this.stageAttemptNumber = stageAttemptNumber; + public RssPartitionToShuffleServerRequest(int shuffleId) { + this.shuffleId = shuffleId; - this.isWritePhase = isWritePhase; } public int getShuffleId() { return shuffleId; } - public int getStageAttemptNumber() { - return stageAttemptNumber; - } - public RssProtos.PartitionToShuffleServerRequest toProto() { RssProtos.PartitionToShuffleServerRequest.Builder builder = RssProtos.PartitionToShuffleServerRequest.newBuilder(); - builder.setStageAttemptId(stageAttemptId); - builder.setStageAttemptNumber(stageAttemptNumber); builder.setShuffleId(shuffleId); - builder.setIsWritePhase(isWritePhase); return builder.build(); } } diff --git a/internal-client/src/main/java/org/apache/uniffle/client/request/RssRegisterShuffleRequest.java b/internal-client/src/main/java/org/apache/uniffle/client/request/RssRegisterShuffleRequest.java index a2cac5367f..48ded96d77 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/request/RssRegisterShuffleRequest.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/request/RssRegisterShuffleRequest.java @@ -39,8 +39,6 @@ public class RssRegisterShuffleRequest { private String user; private ShuffleDataDistributionType dataDistributionType; private int maxConcurrencyPerPartitionToWrite; - private int stageAttemptNumber; - private final MergeContext mergeContext; private Map properties; @@ -61,7 +59,6 @@ public RssRegisterShuffleRequest( user, dataDistributionType, maxConcurrencyPerPartitionToWrite, - 0, null, Collections.emptyMap()); } @@ -74,7 +71,6 @@ public RssRegisterShuffleRequest( String user, ShuffleDataDistributionType dataDistributionType, int maxConcurrencyPerPartitionToWrite, - int stageAttemptNumber, MergeContext mergeContext, Map properties) { this.appId = appId; @@ -84,7 +80,6 @@ public RssRegisterShuffleRequest( this.user = user; this.dataDistributionType = dataDistributionType; this.maxConcurrencyPerPartitionToWrite = maxConcurrencyPerPartitionToWrite; - this.stageAttemptNumber = stageAttemptNumber; this.mergeContext = mergeContext; this.properties = properties; } @@ -105,7 +100,6 @@ public RssRegisterShuffleRequest( user, dataDistributionType, RssClientConf.MAX_CONCURRENCY_PER_PARTITION_TO_WRITE.defaultValue(), - 0, null, Collections.emptyMap()); } @@ -120,7 +114,6 @@ public RssRegisterShuffleRequest( StringUtils.EMPTY, ShuffleDataDistributionType.NORMAL, RssClientConf.MAX_CONCURRENCY_PER_PARTITION_TO_WRITE.defaultValue(), - 0, null, Collections.emptyMap()); } @@ -153,10 +146,6 @@ public int getMaxConcurrencyPerPartitionToWrite() { return maxConcurrencyPerPartitionToWrite; } - public int getStageAttemptNumber() { - return stageAttemptNumber; - } - public MergeContext getMergeContext() { return mergeContext; } diff --git a/internal-client/src/main/java/org/apache/uniffle/client/request/RssReportShuffleFetchFailureRequest.java b/internal-client/src/main/java/org/apache/uniffle/client/request/RssReportShuffleFetchFailureRequest.java index d9cea576a3..5d118cd958 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/request/RssReportShuffleFetchFailureRequest.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/request/RssReportShuffleFetchFailureRequest.java @@ -22,15 +22,25 @@ public class RssReportShuffleFetchFailureRequest { private String appId; private int shuffleId; + private int uniffleShuffleId; private int stageAttemptId; + private int stageAttemptNumber; private int partitionId; private String exception; public RssReportShuffleFetchFailureRequest( - String appId, int shuffleId, int stageAttemptId, int partitionId, String exception) { + String appId, + int shuffleId, + int uniffleShuffleId, + int stageAttemptId, + int stageAttemptNumber, + int partitionId, + String exception) { this.appId = appId; this.shuffleId = shuffleId; + this.uniffleShuffleId = uniffleShuffleId; this.stageAttemptId = stageAttemptId; + this.stageAttemptNumber = stageAttemptNumber; this.partitionId = partitionId; this.exception = exception; } @@ -41,7 +51,9 @@ public ReportShuffleFetchFailureRequest toProto() { builder .setAppId(appId) .setShuffleId(shuffleId) + .setUniffleShuffleId(uniffleShuffleId) .setStageAttemptId(stageAttemptId) + .setStageAttemptNumber(stageAttemptNumber) .setPartitionId(partitionId); if (exception != null) { builder.setException(exception); diff --git a/internal-client/src/main/java/org/apache/uniffle/client/request/RssReportShuffleWriteFailureRequest.java b/internal-client/src/main/java/org/apache/uniffle/client/request/RssReportShuffleWriteFailureRequest.java deleted file mode 100644 index 56da7c6c33..0000000000 --- a/internal-client/src/main/java/org/apache/uniffle/client/request/RssReportShuffleWriteFailureRequest.java +++ /dev/null @@ -1,74 +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.client.request; - -import java.util.List; - -import com.google.common.collect.Lists; - -import org.apache.uniffle.common.ShuffleServerInfo; -import org.apache.uniffle.proto.RssProtos.ReportShuffleWriteFailureRequest; -import org.apache.uniffle.proto.RssProtos.ShuffleServerId; - -public class RssReportShuffleWriteFailureRequest { - private String appId; - private int shuffleId; - private int stageAttemptId; - private int stageAttemptNumber; - private List shuffleServerInfos; - private String exception; - - public RssReportShuffleWriteFailureRequest( - String appId, - int shuffleId, - int stageAttemptId, - int stageAttemptNumber, - List shuffleServerInfos, - String exception) { - this.appId = appId; - this.shuffleId = shuffleId; - this.stageAttemptId = stageAttemptId; - this.stageAttemptNumber = stageAttemptNumber; - this.shuffleServerInfos = shuffleServerInfos; - this.exception = exception; - } - - public ReportShuffleWriteFailureRequest toProto() { - List shuffleServerIds = Lists.newArrayList(); - for (ShuffleServerInfo shuffleServerInfo : shuffleServerInfos) { - shuffleServerIds.add( - ShuffleServerId.newBuilder() - .setId(shuffleServerInfo.getId()) - .setPort(shuffleServerInfo.getGrpcPort()) - .setIp(shuffleServerInfo.getHost()) - .build()); - } - ReportShuffleWriteFailureRequest.Builder builder = - ReportShuffleWriteFailureRequest.newBuilder(); - builder - .setAppId(appId) - .setShuffleId(shuffleId) - .setStageAttemptId(stageAttemptId) - .setStageAttemptNumber(stageAttemptNumber) - .addAllShuffleServerIds(shuffleServerIds); - if (exception != null) { - builder.setException(exception); - } - return builder.build(); - } -} diff --git a/internal-client/src/main/java/org/apache/uniffle/client/request/RssSendShuffleDataRequest.java b/internal-client/src/main/java/org/apache/uniffle/client/request/RssSendShuffleDataRequest.java index 1b5fdcff80..8fbf18f29c 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/request/RssSendShuffleDataRequest.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/request/RssSendShuffleDataRequest.java @@ -25,7 +25,6 @@ public class RssSendShuffleDataRequest { private String appId; - private int stageAttemptNumber; private int retryMax; private long retryIntervalMax; private Map>> shuffleIdToBlocks; @@ -35,20 +34,10 @@ public RssSendShuffleDataRequest( int retryMax, long retryIntervalMax, Map>> shuffleIdToBlocks) { - this(appId, 0, retryMax, retryIntervalMax, shuffleIdToBlocks); - } - - public RssSendShuffleDataRequest( - String appId, - int stageAttemptNumber, - int retryMax, - long retryIntervalMax, - Map>> shuffleIdToBlocks) { this.appId = appId; this.retryMax = retryMax; this.retryIntervalMax = retryIntervalMax; this.shuffleIdToBlocks = shuffleIdToBlocks; - this.stageAttemptNumber = stageAttemptNumber; } public String getAppId() { @@ -63,10 +52,6 @@ public long getRetryIntervalMax() { return retryIntervalMax; } - public int getStageAttemptNumber() { - return stageAttemptNumber; - } - public Map>> getShuffleIdToBlocks() { return shuffleIdToBlocks; } diff --git a/internal-client/src/main/java/org/apache/uniffle/client/response/RssReassignOnStageRetryResponse.java b/internal-client/src/main/java/org/apache/uniffle/client/response/RssAppUniffleShuffleIdResponse.java similarity index 51% rename from internal-client/src/main/java/org/apache/uniffle/client/response/RssReassignOnStageRetryResponse.java rename to internal-client/src/main/java/org/apache/uniffle/client/response/RssAppUniffleShuffleIdResponse.java index 3762ea41bb..db2c9877f7 100644 --- a/internal-client/src/main/java/org/apache/uniffle/client/response/RssReassignOnStageRetryResponse.java +++ b/internal-client/src/main/java/org/apache/uniffle/client/response/RssAppUniffleShuffleIdResponse.java @@ -17,29 +17,21 @@ package org.apache.uniffle.client.response; -import org.apache.uniffle.common.rpc.StatusCode; import org.apache.uniffle.proto.RssProtos; -public class RssReassignOnStageRetryResponse extends ClientResponse { - private RssProtos.StageAttemptShuffleHandleInfo shuffleHandleInfoProto; +public class RssAppUniffleShuffleIdResponse { + private final int generatorShuffleId; - public RssReassignOnStageRetryResponse( - StatusCode statusCode, - String message, - RssProtos.StageAttemptShuffleHandleInfo shuffleHandleInfoProto) { - super(statusCode, message); - this.shuffleHandleInfoProto = shuffleHandleInfoProto; + public RssAppUniffleShuffleIdResponse(int generatorShuffleId) { + this.generatorShuffleId = generatorShuffleId; } - public RssProtos.StageAttemptShuffleHandleInfo getShuffleHandleInfoProto() { - return shuffleHandleInfoProto; + public int getGeneratorShuffleId() { + return generatorShuffleId; } - public static RssReassignOnStageRetryResponse fromProto( - RssProtos.ReassignOnStageRetryResponse response) { - return new RssReassignOnStageRetryResponse( - StatusCode.valueOf(response.getStatus().name()), - response.getMsg(), - response.getShuffleHandleInfo()); + public static RssAppUniffleShuffleIdResponse fromProto( + RssProtos.AppUniffleShuffleIdResponse response) { + return new RssAppUniffleShuffleIdResponse(response.getGeneratorShuffleId()); } } diff --git a/internal-client/src/main/java/org/apache/uniffle/client/response/RssReportShuffleWriteFailureResponse.java b/internal-client/src/main/java/org/apache/uniffle/client/response/RssReportShuffleWriteFailureResponse.java deleted file mode 100644 index 4f4d7b3be3..0000000000 --- a/internal-client/src/main/java/org/apache/uniffle/client/response/RssReportShuffleWriteFailureResponse.java +++ /dev/null @@ -1,53 +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.client.response; - -import org.apache.uniffle.common.rpc.StatusCode; -import org.apache.uniffle.proto.RssProtos.ReportShuffleWriteFailureResponse; - -public class RssReportShuffleWriteFailureResponse extends ClientResponse { - private boolean reSubmitWholeStage; - - public RssReportShuffleWriteFailureResponse(StatusCode code, String msg, boolean recomputeStage) { - super(code, msg); - this.reSubmitWholeStage = recomputeStage; - } - - public boolean getReSubmitWholeStage() { - return this.reSubmitWholeStage; - } - - public ReportShuffleWriteFailureResponse toProto() { - ReportShuffleWriteFailureResponse.Builder builder = - ReportShuffleWriteFailureResponse.newBuilder(); - return builder - .setStatus(getStatusCode().toProto()) - .setMsg(getMessage()) - .setReSubmitWholeStage(reSubmitWholeStage) - .build(); - } - - public static RssReportShuffleWriteFailureResponse fromProto( - ReportShuffleWriteFailureResponse response) { - return new RssReportShuffleWriteFailureResponse( - // todo: [issue#780] add fromProto for StatusCode issue - StatusCode.valueOf(response.getStatus().name()), - response.getMsg(), - response.getReSubmitWholeStage()); - } -} diff --git a/proto/src/main/proto/Rss.proto b/proto/src/main/proto/Rss.proto index 4e53175872..4686616619 100644 --- a/proto/src/main/proto/Rss.proto +++ b/proto/src/main/proto/Rss.proto @@ -199,9 +199,8 @@ message ShuffleRegisterRequest { string user = 5; DataDistribution shuffleDataDistribution = 6; int32 maxConcurrencyPerPartitionToWrite = 7; - int32 stageAttemptNumber = 8; - MergeContext mergeContext = 9; - map properties = 10; + MergeContext mergeContext = 8; + map properties = 9; } enum DataDistribution { @@ -239,7 +238,6 @@ message SendShuffleDataRequest { int64 requireBufferId = 3; repeated ShuffleData shuffleData = 4; int64 timestamp = 5; - int32 stageAttemptNumber = 6; } message SendShuffleDataResponse { @@ -455,9 +453,6 @@ message GetShuffleServerRequest { int32 assignmentShuffleServerNumber = 10; int32 estimateTaskConcurrency = 11; repeated string faultyServerIds = 12; - int32 stageId = 13; - int32 stageAttemptNumber = 14; - bool reassign = 15; } message PartitionRangeAssignment { @@ -573,13 +568,11 @@ message CancelDecommissionResponse { // per application. service ShuffleManager { rpc reportShuffleFetchFailure (ReportShuffleFetchFailureRequest) returns (ReportShuffleFetchFailureResponse); - // Gets the mapping between partitions and ShuffleServer from the ShuffleManager server on Stage Retry. - rpc getPartitionToShufflerServerWithStageRetry(PartitionToShuffleServerRequest) returns (ReassignOnStageRetryResponse); // Gets the mapping between partitions and ShuffleServer from the ShuffleManager server on Block Retry. - rpc getPartitionToShufflerServerWithBlockRetry(PartitionToShuffleServerRequest) returns (ReassignOnBlockSendFailureResponse); - // Report write failures to ShuffleManager - rpc reportShuffleWriteFailure (ReportShuffleWriteFailureRequest) returns (ReportShuffleWriteFailureResponse); - // Reassign on block send failure that occurs in writer + rpc getPartitionToShufflerServer(PartitionToShuffleServerRequest) returns (ReassignOnBlockSendFailureResponse); + // Get the regenerated ShuffleId from the Driver. + rpc getUniffleShuffleId(AppUniffleShuffleIdRequest) returns (AppUniffleShuffleIdResponse); +// Reassign on block send failure that occurs in writer rpc reassignOnBlockSendFailure(RssReassignOnBlockSendFailureRequest) returns (ReassignOnBlockSendFailureResponse); rpc reportShuffleResult (ReportShuffleResultRequest) returns (ReportShuffleResultResponse); rpc getShuffleResult (GetShuffleResultRequest) returns (GetShuffleResultResponse); @@ -589,6 +582,29 @@ service ShuffleManager { rpc reportShuffleReadMetric (ReportShuffleReadMetricRequest) returns (ReportShuffleReadMetricResponse); } +message AppUniffleShuffleIdRequest { + int32 shuffleId = 1; + string appShuffleIdentifier = 2; + bool isWriter = 3; +} + +message AppUniffleShuffleIdResponse { + int32 generatorShuffleId = 1; +} + +message ReassignServersRequest{ + int32 stageId = 1; + int32 stageAttemptNumber = 2; + int32 shuffleId = 3; + int32 numPartitions = 4; +} + +message ReassignServersResponse { + StatusCode status = 1; + bool needReassign = 2; + string msg = 3; +} + message ReportShuffleWriteMetricRequest { int32 shuffleId = 1; int32 stageId = 2; @@ -640,6 +656,8 @@ message ReportShuffleFetchFailureRequest { string exception = 5; // todo: report ShuffleServerId if needed // ShuffleServerId serverId = 6; + int32 uniffleShuffleId = 6; + int32 stageAttemptNumber = 7; } message ReportShuffleFetchFailureResponse { @@ -649,22 +667,9 @@ message ReportShuffleFetchFailureResponse { } message PartitionToShuffleServerRequest { - int32 stageAttemptId = 1; - int32 stageAttemptNumber = 2; - int32 shuffleId = 3; - bool isWritePhase = 4; -} - -message ReassignOnStageRetryResponse { - StatusCode status = 1; - string msg = 2; - StageAttemptShuffleHandleInfo shuffleHandleInfo = 3; + int32 shuffleId = 1; } -message StageAttemptShuffleHandleInfo { - repeated MutableShuffleHandleInfo historyMutableShuffleHandleInfo= 1; - MutableShuffleHandleInfo currentMutableShuffleHandleInfo = 2; -} message MutableShuffleHandleInfo { int32 shuffleId = 1; @@ -698,21 +703,6 @@ message RemoteStorageInfo{ map confItems = 2; } -message ReportShuffleWriteFailureRequest { - string appId = 1; - int32 shuffleId = 2; - int32 stageAttemptId = 3; - int32 stageAttemptNumber = 4; - repeated ShuffleServerId shuffleServerIds= 5; - string exception = 6; -} - -message ReportShuffleWriteFailureResponse { - StatusCode status = 1; - bool reSubmitWholeStage = 2; - string msg = 3; -} - message RssReassignOnBlockSendFailureRequest { int32 shuffleId = 1; map failurePartitionToServerIds = 2; 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 411aeb4b33..ad540487ff 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleServerGrpcService.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleServerGrpcService.java @@ -97,7 +97,6 @@ import org.apache.uniffle.proto.RssProtos.ShuffleRegisterResponse; import org.apache.uniffle.proto.ShuffleServerGrpc.ShuffleServerImplBase; import org.apache.uniffle.server.audit.ServerRpcAuditContext; -import org.apache.uniffle.server.block.ShuffleBlockIdManager; import org.apache.uniffle.server.buffer.PreAllocatedBufferInfo; import org.apache.uniffle.server.merge.MergeStatus; import org.apache.uniffle.storage.common.Storage; @@ -247,92 +246,12 @@ public void unregisterShuffle( public void registerShuffle( ShuffleRegisterRequest req, StreamObserver responseObserver) { try (ServerRpcAuditContext auditContext = createAuditContext("registerShuffle")) { - ShuffleRegisterResponse reply; String appId = req.getAppId(); int shuffleId = req.getShuffleId(); - int stageAttemptNumber = req.getStageAttemptNumber(); String remoteStoragePath = req.getRemoteStorage().getPath(); String user = req.getUser(); auditContext.withAppId(appId).withShuffleId(shuffleId); - auditContext.withArgs( - "remoteStoragePath=" - + remoteStoragePath - + ", user=" - + user - + ", stageAttemptNumber=" - + stageAttemptNumber); - // If the Stage is registered for the first time, you do not need to consider the Stage retry - // and delete the Block data that has been sent. - ShuffleTaskInfo taskInfo = shuffleServer.getShuffleTaskManager().getShuffleTaskInfo(appId); - if (taskInfo != null) { - // Prevents AttemptNumber of multiple stages from modifying the latest AttemptNumber. - synchronized (taskInfo) { - int lastAttemptNumber = taskInfo.getLatestStageAttemptNumber(shuffleId); - if (stageAttemptNumber > 0 && stageAttemptNumber > lastAttemptNumber) { - taskInfo.refreshLatestStageAttemptNumber(shuffleId, stageAttemptNumber); - try { - long start = System.currentTimeMillis(); - shuffleServer - .getShuffleTaskManager() - .removeShuffleDataSyncRenameAndDelete(appId, shuffleId); - LOG.info( - "Deleted the previous stage attempt data due to stage recomputing for app: {}, " - + "shuffleId: {}, stageAttemptNumber: {}. It costs {} ms", - appId, - shuffleId, - lastAttemptNumber, - System.currentTimeMillis() - start); - // Add a check to prevent undeleted metadata. - ShuffleBlockIdManager shuffleBlockIdManager = - shuffleServer - .getShuffleTaskManager() - .getShuffleTaskInfo(appId) - .getShuffleBlockIdManager(); - long blockCountByShuffleId = - shuffleBlockIdManager.getBlockCountByShuffleId( - appId, Lists.newArrayList(shuffleId)); - if (blockCountByShuffleId != 0) { - LOG.error( - "Metadata is not deleted on clearing previous stage attempt data for app: {}, shuffleId: {}, stageAttemptNumber: {}", - appId, - shuffleId, - lastAttemptNumber); - StatusCode code = StatusCode.INTERNAL_ERROR; - auditContext.withStatusCode(code); - reply = ShuffleRegisterResponse.newBuilder().setStatus(code.toProto()).build(); - responseObserver.onNext(reply); - responseObserver.onCompleted(); - return; - } - } catch (Exception e) { - LOG.error( - "Errors on clearing previous stage attempt data for app: {}, shuffleId: {}, stageAttemptNumber: {}", - appId, - shuffleId, - lastAttemptNumber, - e); - StatusCode code = StatusCode.INTERNAL_ERROR; - auditContext.withStatusCode(code); - reply = ShuffleRegisterResponse.newBuilder().setStatus(code.toProto()).build(); - responseObserver.onNext(reply); - responseObserver.onCompleted(); - return; - } - } else if (stageAttemptNumber > 0 && stageAttemptNumber <= lastAttemptNumber) { - LOG.info( - "The registration failed. The latest retry count is smaller than the existing retry count. This situation should not exist."); - // When a Stage retry occurs, the first or last registration of a Stage may need to be - // ignored and the ignored status quickly returned. - StatusCode code = StatusCode.STAGE_RETRY_IGNORE; - auditContext.withStatusCode(code); - reply = ShuffleRegisterResponse.newBuilder().setStatus(code.toProto()).build(); - responseObserver.onNext(reply); - responseObserver.onCompleted(); - return; - } - } - } - + auditContext.withArgs("remoteStoragePath=" + remoteStoragePath + ", user=" + user); ShuffleDataDistributionType shuffleDataDistributionType = ShuffleDataDistributionType.valueOf( Optional.ofNullable(req.getShuffleDataDistribution()) @@ -348,15 +267,11 @@ public void registerShuffle( List partitionRanges = toPartitionRanges(req.getPartitionRangesList()); LOG.info( - "Get register request for appId[" - + appId - + "], shuffleId[" - + shuffleId - + "], remoteStorage[" - + remoteStoragePath - + "] with " - + partitionRanges.size() - + " partition ranges. User: {}", + "Get register request for appId[{}], shuffleId[{}], remoteStorage[{}] with {} partition ranges. User: {}", + appId, + shuffleId, + remoteStoragePath, + partitionRanges.size(), user); StatusCode result = @@ -365,7 +280,6 @@ public void registerShuffle( .registerShuffle( appId, shuffleId, - stageAttemptNumber, partitionRanges, new RemoteStorageInfo(remoteStoragePath, remoteStorageConf), user, @@ -383,7 +297,6 @@ public void registerShuffle( .registerShuffle( appId + MERGE_APP_SUFFIX, shuffleId, - stageAttemptNumber, partitionRanges, new RemoteStorageInfo(remoteStoragePath, remoteStorageConf), user, @@ -398,7 +311,8 @@ public void registerShuffle( } } auditContext.withStatusCode(result); - reply = ShuffleRegisterResponse.newBuilder().setStatus(result.toProto()).build(); + ShuffleRegisterResponse reply = + ShuffleRegisterResponse.newBuilder().setStatus(result.toProto()).build(); responseObserver.onNext(reply); responseObserver.onCompleted(); } @@ -413,7 +327,6 @@ public void sendShuffleData( int shuffleId = req.getShuffleId(); long requireBufferId = req.getRequireBufferId(); long timestamp = req.getTimestamp(); - int stageAttemptNumber = req.getStageAttemptNumber(); auditContext.withAppId(appId).withShuffleId(shuffleId); auditContext.withArgs( @@ -421,8 +334,6 @@ public void sendShuffleData( + requireBufferId + ", timestamp=" + timestamp - + ", stageAttemptNumber=" - + stageAttemptNumber + ", shuffleDataSize=" + req.getShuffleDataCount()); @@ -448,21 +359,6 @@ public void sendShuffleData( responseObserver.onCompleted(); return; } - Integer latestStageAttemptNumber = taskInfo.getLatestStageAttemptNumber(shuffleId); - // The Stage retry occurred, and the task before StageNumber was simply ignored and not - // processed if the task was being sent. - if (stageAttemptNumber < latestStageAttemptNumber) { - String responseMessage = "A retry has occurred at the Stage, sending data is invalid."; - reply = - SendShuffleDataResponse.newBuilder() - .setStatus(StatusCode.STAGE_RETRY_IGNORE.toProto()) - .setRetMsg(responseMessage) - .build(); - auditContext.withStatusCode(StatusCode.fromProto(reply.getStatus())); - responseObserver.onNext(reply); - responseObserver.onCompleted(); - return; - } if (timestamp > 0) { /* * Here we record the transport time, but we don't consider the impact of data size on transport time. diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleTaskInfo.java b/server/src/main/java/org/apache/uniffle/server/ShuffleTaskInfo.java index 687622bb07..4c8b32829e 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleTaskInfo.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleTaskInfo.java @@ -81,7 +81,6 @@ public class ShuffleTaskInfo { /** shuffleId -> shuffleDetailInfo */ private final Map shuffleDetailInfos; - private final Map latestStageAttemptNumbers; private Map properties; private ShuffleBlockIdManager shuffleBlockIdManager; @@ -97,7 +96,6 @@ public ShuffleTaskInfo(String appId) { this.existHugePartition = new AtomicBoolean(false); this.specification = new AtomicReference<>(); this.partitionBlockCounters = JavaUtils.newConcurrentMap(); - this.latestStageAttemptNumbers = JavaUtils.newConcurrentMap(); this.shuffleDetailInfos = JavaUtils.newConcurrentMap(); } @@ -283,14 +281,6 @@ public long getBlockNumber(int shuffleId, int partitionId) { return counter.get(); } - public Integer getLatestStageAttemptNumber(int shuffleId) { - return latestStageAttemptNumbers.getOrDefault(shuffleId, 0); - } - - public void refreshLatestStageAttemptNumber(int shuffleId, int stageAttemptNumber) { - latestStageAttemptNumbers.put(shuffleId, stageAttemptNumber); - } - public PartitionInfo getMaxSizePartitionInfo() { return maxSizePartitionInfo; } diff --git a/server/src/main/java/org/apache/uniffle/server/ShuffleTaskManager.java b/server/src/main/java/org/apache/uniffle/server/ShuffleTaskManager.java index c460b4c099..08180baa0e 100644 --- a/server/src/main/java/org/apache/uniffle/server/ShuffleTaskManager.java +++ b/server/src/main/java/org/apache/uniffle/server/ShuffleTaskManager.java @@ -302,7 +302,6 @@ public StatusCode registerShuffle( return registerShuffle( appId, shuffleId, - 0, partitionRanges, remoteStorageInfo, user, @@ -314,7 +313,6 @@ public StatusCode registerShuffle( public StatusCode registerShuffle( String appId, int shuffleId, - int stageAttemptNumber, List partitionRanges, RemoteStorageInfo remoteStorageInfo, String user, @@ -335,7 +333,6 @@ public StatusCode registerShuffle( .dataDistributionType(dataDistType) .build()); taskInfo.setShuffleBlockIdManagerIfNeeded(shuffleBlockIdManager); - taskInfo.refreshLatestStageAttemptNumber(shuffleId, stageAttemptNumber); taskInfo.getShuffleBlockIdManager().registerAppId(appId); for (PartitionRange partitionRange : partitionRanges) { shuffleBufferManager.registerBuffer( 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 ab6ce2834e..a20a08f129 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 @@ -206,36 +206,6 @@ public void handleSendShuffleDataRequest(TransportClient client, SendShuffleData client.getChannel().writeAndFlush(rpcResponse); return; } - Integer latestStageAttemptNumber = taskInfo.getLatestStageAttemptNumber(shuffleId); - // The Stage retry occurred, and the task before StageNumber was simply ignored and not - // processed if the task was being sent. - if (stageAttemptNumber < latestStageAttemptNumber) { - String responseMessage = "A retry has occurred at the Stage, sending data is invalid."; - rpcResponse = - new RpcResponse(req.getRequestId(), StatusCode.STAGE_RETRY_IGNORE, responseMessage); - LOG.warn( - "Stage retry occurred, appId[" - + appId - + "], shuffleId[" - + shuffleId - + "], stageAttemptNumber[" - + stageAttemptNumber - + "], latestStageAttemptNumber[" - + latestStageAttemptNumber - + "]"); - releaseNettyBufferAndMetrics( - req, - appId, - shuffleId, - requireBufferId, - requireBlocksSize, - shuffleBufferManager, - info, - isPreAllocated); - auditContext.withStatusCode(rpcResponse.getStatusCode()); - client.getChannel().writeAndFlush(rpcResponse); - return; - } long timestamp = req.getTimestamp(); if (timestamp > 0) { /* diff --git a/server/src/test/java/org/apache/uniffle/server/MockedShuffleServerGrpcService.java b/server/src/test/java/org/apache/uniffle/server/MockedShuffleServerGrpcService.java index 5a81b3b83f..db52033a1e 100644 --- a/server/src/test/java/org/apache/uniffle/server/MockedShuffleServerGrpcService.java +++ b/server/src/test/java/org/apache/uniffle/server/MockedShuffleServerGrpcService.java @@ -140,12 +140,6 @@ public void sendShuffleData( responseObserver.onCompleted(); return; } - if (mockSendDataFailedStageNumber == request.getStageAttemptNumber()) { - LOG.info( - "Add a mocked sendData failed on sendShuffleData with the stage number={}", - mockSendDataFailedStageNumber); - throw new RuntimeException("This write request is failed as mocked failure!"); - } if (mockedTimeout > 0) { LOG.info("Add a mocked timeout on sendShuffleData"); Uninterruptibles.sleepUninterruptibly(mockedTimeout, TimeUnit.MILLISECONDS);