maobaolong commented on code in PR #2408: URL: https://github.com/apache/uniffle/pull/2408#discussion_r2009119791
########## common/src/main/java/org/apache/uniffle/common/PartitionSplitMode.java: ########## @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.uniffle.common; + +public enum PartitionSplitMode { Review Comment: It would be clear for new developer if you add comments here for these two mode. ########## client-spark/spark3/src/main/java/org/apache/spark/shuffle/writer/RssShuffleWriter.java: ########## @@ -632,22 +632,69 @@ private void reassignOnPartitionNeedSplit(FailedBlockSendTracker failedTracker) failedTracker .removeAllTrackedPartitions() .forEach( - partitionStatus -> - failurePartitionToServers - .computeIfAbsent(partitionStatus.getPartitionId(), x -> new ArrayList<>()) - .add( - new ReceivingFailureServer( - partitionStatus.getShuffleServerInfo().getId(), StatusCode.SUCCESS))); - if (!failurePartitionToServers.isEmpty()) { - doReassignOnBlockSendFailure(failurePartitionToServers, true); + partitionStatus -> { + List<ReceivingFailureServer> servers = + failurePartitionToServers.computeIfAbsent( + partitionStatus.getPartitionId(), x -> new ArrayList<>()); + String serverId = partitionStatus.getShuffleServerInfo().getId(); + // todo: use better data structure to filter + if (!servers.stream() + .map(x -> x.getServerId()) + .collect(Collectors.toSet()) + .contains(serverId)) { + servers.add(new ReceivingFailureServer(serverId, StatusCode.SUCCESS)); + } + }); + + if (failurePartitionToServers.isEmpty()) { + return; + } + + // + // For the [load balance] mode + // Once partition has been split, the following split trigger will be ignored. + // + // For the [pipeline] mode + // The split request will be always response + // + Map<Integer, List<ReceivingFailureServer>> partitionToServersReassignList = new HashMap<>(); + for (Map.Entry<Integer, List<ReceivingFailureServer>> entry : + failurePartitionToServers.entrySet()) { + int partitionId = entry.getKey(); + boolean isSkip = taskAttemptAssignment.isSkipPartitionSplit(partitionId); + if (!isSkip) { + partitionToServersReassignList.put(partitionId, entry.getValue()); + } } + + if (partitionToServersReassignList.isEmpty()) { + LOG.info( + "[Partition split] Skip the following partition split request (maybe has been load balanced). partitionIds: {}", + failurePartitionToServers.keySet()); + return; + } + + doReassignOnBlockSendFailure(partitionToServersReassignList, true); + + LOG.info("========================= Partition Split Result ========================="); Review Comment: Is this should be removed or set to debug mode? ########## common/src/main/java/org/apache/uniffle/common/config/RssClientConf.java: ########## @@ -271,6 +272,19 @@ public class RssClientConf { .withDescription( "Whether to support rss client block send failure retry, default value is false."); + public static final ConfigOption<PartitionSplitMode> RSS_CLIENT_PARTITION_SPLIT_MODE = + ConfigOptions.key("rss.client.reassign.partitionSplitMode") + .enumType(PartitionSplitMode.class) + .defaultValue(PartitionSplitMode.PIPELINE) + .withDescription("The partition split mode. default is PIPELINE."); + + public static final ConfigOption<Integer> RSS_CLIENT_PARTITION_SPLIT_LOAD_BALANCE_SERVER_NUMBER = + ConfigOptions.key("rss.client.reassign.partitionSplitLoadBalanceServerNumber") + .intType() + .defaultValue(10) Review Comment: The default `10` maybe too big for the small cluster, is there a way to define a ratio of cluster's worker number? For example, 0.5 means half of works. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
