Skip to content

Commit

Permalink
[apache#1608][part-3] feat(spark3): support reading from partition bl…
Browse files Browse the repository at this point in the history
…ock data reassignment servers
  • Loading branch information
zuston committed Apr 8, 2024
1 parent 80caa0e commit befa87a
Show file tree
Hide file tree
Showing 5 changed files with 166 additions and 12 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,9 @@
package org.apache.spark.shuffle;

import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
Expand Down Expand Up @@ -114,4 +116,28 @@ public ShuffleServerInfo createNewReassignmentForMultiPartitions(
}
return replacement;
}

/** This composes the partition's replica server + replacement servers. */
public Map<Integer, List<ShuffleServerInfo>> listAllPartitionAssignmentServers() {
Map<Integer, List<ShuffleServerInfo>> partitionServer = new HashMap<>();
for (Map.Entry<Integer, List<ShuffleServerInfo>> entry : partitionToServers.entrySet()) {
int partitionId = entry.getKey();
List<ShuffleServerInfo> replicas = entry.getValue();
Map<Integer, ShuffleServerInfo> replacements = failoverPartitionServers.get(partitionId);
if (replacements == null) {
replacements = Collections.emptyMap();
}

List<ShuffleServerInfo> servers =
partitionServer.computeIfAbsent(partitionId, k -> new ArrayList<>());
for (int i = 0; i < replicas.size(); i++) {
servers.add(replicas.get(i));
ShuffleServerInfo replacement = replacements.get(i);
if (replacement != null) {
servers.add(replacement);
}
}
}
return partitionServer;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,8 +35,8 @@

public class ShuffleHandleInfoTest {

private ShuffleServerInfo createFakeServerInfo(String host) {
return new ShuffleServerInfo(host, 1);
private ShuffleServerInfo createFakeServerInfo(String id) {
return new ShuffleServerInfo(id, "1.1.1.1", 1);
}

@Test
Expand All @@ -58,4 +58,25 @@ public void testReassignment() {

assertEquals(newServer, handleInfo.useExistingReassignmentForMultiPartitions(partitions, "a"));
}

@Test
public void testListAllPartitionAssignmentServers() {
Map<Integer, List<ShuffleServerInfo>> partitionToServers = new HashMap<>();
partitionToServers.put(1, Arrays.asList(createFakeServerInfo("a"), createFakeServerInfo("b")));
partitionToServers.put(2, Arrays.asList(createFakeServerInfo("c")));

ShuffleHandleInfo handleInfo =
new ShuffleHandleInfo(1, partitionToServers, new RemoteStorageInfo(""));

Set<Integer> partitions = new HashSet<>();
partitions.add(2);
handleInfo.createNewReassignmentForMultiPartitions(partitions, "c", createFakeServerInfo("d"));

Map<Integer, List<ShuffleServerInfo>> partitionAssignment =
handleInfo.listAllPartitionAssignmentServers();
assertEquals(2, partitionAssignment.size());
assertEquals(
Arrays.asList(createFakeServerInfo("c"), createFakeServerInfo("d")),
partitionAssignment.get(2));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -663,19 +663,13 @@ public <K, C> ShuffleReader<K, C> getReaderImpl(
rssShuffleHandle.getPartitionToServers(),
rssShuffleHandle.getRemoteStorage());
}
Map<Integer, List<ShuffleServerInfo>> allPartitionToServers =
shuffleHandleInfo.getPartitionToServers();
Map<Integer, List<ShuffleServerInfo>> requirePartitionToServers =
allPartitionToServers.entrySet().stream()
.filter(x -> x.getKey() >= startPartition && x.getKey() < endPartition)
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
Map<ShuffleServerInfo, Set<Integer>> serverToPartitions =
RssUtils.generateServerToPartitions(requirePartitionToServers);
Map<ShuffleServerInfo, Set<Integer>> blockIdServerToPartitions =
getBlockIdServers(shuffleHandleInfo, startPartition, endPartition);
long start = System.currentTimeMillis();
Roaring64NavigableMap blockIdBitmap =
getShuffleResultForMultiPart(
clientType,
serverToPartitions,
blockIdServerToPartitions,
rssShuffleHandle.getAppId(),
shuffleId,
context.stageAttemptNumber());
Expand Down Expand Up @@ -721,7 +715,20 @@ public <K, C> ShuffleReader<K, C> getReaderImpl(
readMetrics,
RssSparkConfig.toRssConf(sparkConf),
dataDistributionType,
allPartitionToServers);
shuffleHandleInfo.listAllPartitionAssignmentServers());
}

private Map<ShuffleServerInfo, Set<Integer>> getBlockIdServers(
ShuffleHandleInfo shuffleHandleInfo, int startPartition, int endPartition) {
Map<Integer, List<ShuffleServerInfo>> allPartitionToServers =
shuffleHandleInfo.getPartitionToServers();
Map<Integer, List<ShuffleServerInfo>> requirePartitionToServers =
allPartitionToServers.entrySet().stream()
.filter(x -> x.getKey() >= startPartition && x.getKey() < endPartition)
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
Map<ShuffleServerInfo, Set<Integer>> serverToPartitions =
RssUtils.generateServerToPartitions(requirePartitionToServers);
return serverToPartitions;
}

@SuppressFBWarnings("REC_CATCH_EXCEPTION")
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,96 @@
/*
* 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.Map;

import com.google.common.collect.Maps;
import org.apache.spark.SparkConf;
import org.apache.spark.shuffle.RssSparkConfig;
import org.junit.jupiter.api.BeforeAll;
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.ShuffleServer;
import org.apache.uniffle.server.ShuffleServerConf;
import org.apache.uniffle.server.buffer.ShuffleBufferManager;
import org.apache.uniffle.storage.util.StorageType;

import static org.apache.uniffle.client.util.RssClientConfig.RSS_CLIENT_ASSIGNMENT_SHUFFLE_SERVER_NUMBER;
import static org.junit.jupiter.api.Assertions.assertEquals;

/** This class is to test the mechanism of partition block data reassignment. */
public class PartitionBlockDataReassignTest extends SparkSQLTest {

private static String basePath;

@BeforeAll
public static void setupServers(@TempDir File tmpDir) throws Exception {
// for coordinator
CoordinatorConf coordinatorConf = getCoordinatorConf();
coordinatorConf.setLong("rss.coordinator.app.expired", 5000);
Map<String, String> dynamicConf = Maps.newHashMap();
dynamicConf.put(RssSparkConfig.RSS_STORAGE_TYPE.key(), StorageType.MEMORY_LOCALFILE.name());
addDynamicConf(coordinatorConf, dynamicConf);
createCoordinatorServer(coordinatorConf);

// for shuffle-server
File dataDir1 = new File(tmpDir, "data1");
File dataDir2 = new File(tmpDir, "data2");
basePath = dataDir1.getAbsolutePath() + "," + dataDir2.getAbsolutePath();

ShuffleServerConf grpcShuffleServerConf1 = buildShuffleServerConf(ServerType.GRPC);
createShuffleServer(grpcShuffleServerConf1);

ShuffleServerConf grpcShuffleServerConf2 = buildShuffleServerConf(ServerType.GRPC);
createShuffleServer(grpcShuffleServerConf2);

startServers();

// simulate one server without enough buffer
ShuffleServer faultyShuffleServer = grpcShuffleServers.get(0);
ShuffleBufferManager bufferManager = faultyShuffleServer.getShuffleBufferManager();
bufferManager.setUsedMemory(bufferManager.getCapacity() + 100);
}

private static ShuffleServerConf buildShuffleServerConf(ServerType serverType) throws Exception {
ShuffleServerConf shuffleServerConf = getShuffleServerConf(serverType);
shuffleServerConf.setLong("rss.server.heartbeat.interval", 5000);
shuffleServerConf.setLong("rss.server.app.expired.withoutHeartbeat", 4000);
shuffleServerConf.setString("rss.storage.basePath", basePath);
shuffleServerConf.setString("rss.storage.type", StorageType.MEMORY_LOCALFILE.name());
return shuffleServerConf;
}

@Override
public void updateRssStorage(SparkConf sparkConf) {
sparkConf.set("spark" + RSS_CLIENT_ASSIGNMENT_SHUFFLE_SERVER_NUMBER, "1");
}

@Override
public void checkShuffleData() throws Exception {
Thread.sleep(12000);
String[] paths = basePath.split(",");
for (String path : paths) {
File f = new File(path);
assertEquals(0, f.list().length);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -712,4 +712,8 @@ public boolean limitHugePartition(
}
return false;
}

public void setUsedMemory(long usedMemory) {
this.usedMemory.set(usedMemory);
}
}

0 comments on commit befa87a

Please sign in to comment.