Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Support writing multi files of single partition to improve speed in HDFS storage #396

Merged
merged 6 commits into from
Dec 13, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions docs/server_guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -84,3 +84,4 @@ This document will introduce how to deploy Uniffle shuffle servers.
|rss.server.disk.capacity|-1|Disk capacity that shuffle server can use. If it's negative, it will use the default disk whole space|
|rss.server.multistorage.fallback.strategy.class|-|The fallback strategy for `MEMORY_LOCALFILE_HDFS`. Support `org.apache.uniffle.server.storage.RotateStorageManagerFallbackStrategy`,`org.apache.uniffle.server.storage.LocalStorageManagerFallbackStrategy` and `org.apache.uniffle.server.storage.HdfsStorageManagerFallbackStrategy`. If not set, `org.apache.uniffle.server.storage.HdfsStorageManagerFallbackStrategy` will be used.|
|rss.server.leak.shuffledata.check.interval|3600000|The interval of leak shuffle data check (ms)|
|rss.server.max.concurrency.of.single.partition.writer|1|The max concurrency of single partition writer, the data partition file number is equal to this value. Default value is 1. This config could improve the writing speed, especially for huge partition.|
Original file line number Diff line number Diff line change
@@ -0,0 +1,152 @@
/*
* 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.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.stream.IntStream;

import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import org.roaringbitmap.longlong.Roaring64NavigableMap;

import org.apache.uniffle.client.impl.ShuffleReadClientImpl;
import org.apache.uniffle.client.request.RssFinishShuffleRequest;
import org.apache.uniffle.client.request.RssRegisterShuffleRequest;
import org.apache.uniffle.client.request.RssSendCommitRequest;
import org.apache.uniffle.client.request.RssSendShuffleDataRequest;
import org.apache.uniffle.client.util.DefaultIdHelper;
import org.apache.uniffle.common.PartitionRange;
import org.apache.uniffle.common.ShuffleBlockInfo;
import org.apache.uniffle.common.ShuffleServerInfo;
import org.apache.uniffle.coordinator.CoordinatorConf;
import org.apache.uniffle.server.ShuffleServerConf;
import org.apache.uniffle.storage.util.StorageType;

import static org.apache.uniffle.common.util.Constants.SHUFFLE_DATA_FILE_SUFFIX;
import static org.junit.jupiter.api.Assertions.assertEquals;

public class ShuffleServerConcurrentWriteOfHdfsTest extends ShuffleServerWithHdfsTest {
private static final int MAX_CONCURRENCY = 3;

@BeforeAll
public static void setupServers() throws Exception {
CoordinatorConf coordinatorConf = getCoordinatorConf();
createCoordinatorServer(coordinatorConf);
ShuffleServerConf shuffleServerConf = getShuffleServerConf();
shuffleServerConf.setString(ShuffleServerConf.RSS_STORAGE_TYPE, StorageType.HDFS.name());
shuffleServerConf.setInteger(ShuffleServerConf.SERVER_MAX_CONCURRENCY_OF_ONE_PARTITION, MAX_CONCURRENCY);
shuffleServerConf.setBoolean(shuffleServerConf.SINGLE_BUFFER_FLUSH_ENABLED, true);
shuffleServerConf.setLong(shuffleServerConf.SINGLE_BUFFER_FLUSH_THRESHOLD, 1024 * 1024L);
createShuffleServer(shuffleServerConf);
startServers();
}

@Test
public void testConcurrentWrite2Hdfs() throws Exception {
String appId = "testConcurrentWrite2Hdfs";
String dataBasePath = HDFS_URI + "rss/test";
advancedxy marked this conversation as resolved.
Show resolved Hide resolved
RssRegisterShuffleRequest rrsr = new RssRegisterShuffleRequest(
appId,
0,
Lists.newArrayList(new PartitionRange(0, 1)),
dataBasePath
);
shuffleServerClient.registerShuffle(rrsr);

List<Roaring64NavigableMap> bitmaps = new ArrayList<>();
Map<Long, byte[]> expectedDataList = new HashMap<>();
IntStream.range(0, 20).forEach(x -> {
Roaring64NavigableMap bitmap = Roaring64NavigableMap.bitmapOf();
bitmaps.add(bitmap);

Map<Long, byte[]> expectedData = Maps.newHashMap();

List<ShuffleBlockInfo> blocks = createShuffleBlockList(
0,
0,
0,
1,
1024 * 1025,
bitmap,
expectedData,
mockSSI
);
expectedDataList.putAll(expectedData);

Map<Integer, List<ShuffleBlockInfo>> partitionToBlocks = Maps.newHashMap();
partitionToBlocks.put(0, blocks);
Map<Integer, Map<Integer, List<ShuffleBlockInfo>>> shuffleToBlocks = Maps.newHashMap();
shuffleToBlocks.put(0, partitionToBlocks);
RssSendShuffleDataRequest rssdr = new RssSendShuffleDataRequest(appId, 3, 1000, shuffleToBlocks);
shuffleServerClient.sendShuffleData(rssdr);
});

RssSendCommitRequest rscr = new RssSendCommitRequest(appId, 0);
shuffleServerClient.sendCommit(rscr);

RssFinishShuffleRequest rfsr = new RssFinishShuffleRequest(appId, 0);
shuffleServerClient.finishShuffle(rfsr);

// Check the concurrent hdfs file creation
FileStatus[] fileStatuses = fs.listStatus(new Path(dataBasePath + "/" + appId + "/0/0-1"));
long actual = Arrays
.stream(fileStatuses)
.filter(x -> x.getPath().getName().endsWith(SHUFFLE_DATA_FILE_SUFFIX))
.count();
assertEquals(MAX_CONCURRENCY, actual);

ShuffleServerInfo ssi = new ShuffleServerInfo(LOCALHOST, SHUFFLE_SERVER_PORT);
Roaring64NavigableMap blocksBitmap = Roaring64NavigableMap.bitmapOf();
bitmaps.stream().forEach(x -> {
Iterator<Long> iterator = x.iterator();
while (iterator.hasNext()) {
blocksBitmap.add(iterator.next());
}
});

ShuffleReadClientImpl readClient = new ShuffleReadClientImpl(
StorageType.HDFS.name(),
appId,
0,
0,
100,
2,
10,
1000,
dataBasePath,
blocksBitmap,
Roaring64NavigableMap.bitmapOf(0),
Lists.newArrayList(ssi),
new Configuration(),
new DefaultIdHelper()
);

validateResult(readClient, expectedDataList, blocksBitmap);
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@

public class ShuffleServerWithHdfsTest extends ShuffleReadWriteBase {

private ShuffleServerGrpcClient shuffleServerClient;
protected ShuffleServerGrpcClient shuffleServerClient;

@BeforeAll
public static void setupServers() throws Exception {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@ public class ShuffleFlushManager {
private final BlockingQueue<PendingShuffleFlushEvent> pendingEvents = Queues.newLinkedBlockingQueue();
private final long pendingEventTimeoutSec;
private int processPendingEventIndex = 0;
private final int maxConcurrencyOfSingleOnePartition;

public ShuffleFlushManager(ShuffleServerConf shuffleServerConf, String shuffleServerId, ShuffleServer shuffleServer,
StorageManager storageManager) {
Expand All @@ -80,6 +81,8 @@ public ShuffleFlushManager(ShuffleServerConf shuffleServerConf, String shuffleSe
retryMax = shuffleServerConf.getInteger(ShuffleServerConf.SERVER_WRITE_RETRY_MAX);
storageType = shuffleServerConf.get(RssBaseConf.RSS_STORAGE_TYPE);
storageDataReplica = shuffleServerConf.get(RssBaseConf.RSS_STORAGE_DATA_REPLICA);
this.maxConcurrencyOfSingleOnePartition =
shuffleServerConf.get(ShuffleServerConf.SERVER_MAX_CONCURRENCY_OF_ONE_PARTITION);

int waitQueueSize = shuffleServerConf.getInteger(
ShuffleServerConf.SERVER_FLUSH_THREAD_POOL_QUEUE_SIZE);
Expand Down Expand Up @@ -200,7 +203,8 @@ private void flushToFile(ShuffleDataFlushEvent event) {
shuffleServerId,
hadoopConf,
storageDataReplica,
user);
user,
maxConcurrencyOfSingleOnePartition);
ShuffleWriteHandler handler = storage.getOrCreateWriteHandler(request);
writeSuccess = storageManager.write(storage, handler, event);
if (writeSuccess) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -314,6 +314,13 @@ public class ShuffleServerConf extends RssBaseConf {
.defaultValue(3600 * 1000L)
.withDescription("the interval of leak shuffle data check");

public static final ConfigOption<Integer> SERVER_MAX_CONCURRENCY_OF_ONE_PARTITION = ConfigOptions
zuston marked this conversation as resolved.
Show resolved Hide resolved
.key("rss.server.max.concurrency.of.single.partition.writer")
.intType()
.defaultValue(1)
.withDescription("The max concurrency of single partition writer, the data partition file number is "
+ "equal to this value. Default value is 1.");

public ShuffleServerConf() {
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Supplier;
import java.util.stream.IntStream;

import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
Expand Down Expand Up @@ -117,6 +118,35 @@ public void hadoopConfTest() {
assertEquals("value", manager.getHadoopConf().get("a.b"));
}

@Test
public void concurrentWrite2HdfsWriteOfSinglePartition() throws Exception {
ShuffleServerConf shuffleServerConf = new ShuffleServerConf();
shuffleServerConf.set(ShuffleServerConf.RSS_STORAGE_BASE_PATH, Collections.emptyList());
shuffleServerConf.setString(ShuffleServerConf.RSS_STORAGE_TYPE, StorageType.HDFS.name());
int maxConcurrency = 3;
shuffleServerConf.setInteger(ShuffleServerConf.SERVER_MAX_CONCURRENCY_OF_ONE_PARTITION, maxConcurrency);

String appId = "concurrentWrite2HdfsWriteOfSinglePartition_appId";
StorageManager storageManager =
StorageManagerFactory.getInstance().createStorageManager(shuffleServerConf);
storageManager.registerRemoteStorage(appId, remoteStorage);
ShuffleFlushManager manager =
new ShuffleFlushManager(shuffleServerConf, "shuffleServerId", mockShuffleServer, storageManager);

IntStream.range(0, 20).forEach(x -> {
ShuffleDataFlushEvent event = createShuffleDataFlushEvent(appId, 1, 1, 1, null);
manager.addToFlushQueue(event);
});
waitForFlush(manager, appId, 1, 10 * 5);

FileStatus[] fileStatuses = fs.listStatus(new Path(HDFS_URI + "/rss/test/" + appId + "/1/1-1"));
long actual = Arrays.stream(fileStatuses).filter(x -> x.getPath().getName().endsWith("data")).count();

assertEquals(maxConcurrency, actual);
actual = Arrays.stream(fileStatuses).filter(x -> x.getPath().getName().endsWith("index")).count();
assertEquals(maxConcurrency, actual);
}

@Test
public void writeTest() throws Exception {
String appId = "writeTest_appId";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.uniffle.storage.handler.api.ServerReadHandler;
import org.apache.uniffle.storage.handler.api.ShuffleWriteHandler;
import org.apache.uniffle.storage.handler.impl.HdfsShuffleWriteHandler;
import org.apache.uniffle.storage.handler.impl.PooledHdfsShuffleWriteHandler;
import org.apache.uniffle.storage.request.CreateShuffleReadHandlerRequest;
import org.apache.uniffle.storage.request.CreateShuffleWriteHandlerRequest;

Expand Down Expand Up @@ -99,16 +100,30 @@ public void updateWriteMetrics(StorageWriteMetrics metrics) {
ShuffleWriteHandler newWriteHandler(CreateShuffleWriteHandlerRequest request) {
try {
String user = request.getUser();
return new HdfsShuffleWriteHandler(
request.getAppId(),
request.getShuffleId(),
request.getStartPartition(),
request.getEndPartition(),
storagePath,
request.getFileNamePrefix(),
conf,
user
);
if (request.getMaxFileNumber() == 1) {
return new HdfsShuffleWriteHandler(
request.getAppId(),
request.getShuffleId(),
request.getStartPartition(),
request.getEndPartition(),
storagePath,
request.getFileNamePrefix(),
conf,
user
);
} else {
return new PooledHdfsShuffleWriteHandler(
request.getAppId(),
request.getShuffleId(),
request.getStartPartition(),
request.getEndPartition(),
storagePath,
request.getFileNamePrefix(),
conf,
user,
request.getMaxFileNumber()
);
}
} catch (Exception e) {
throw new RuntimeException(e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.uniffle.storage.handler.api;

import java.io.IOException;
import java.util.List;

import org.apache.uniffle.common.ShufflePartitionedBlock;
Expand All @@ -28,8 +27,7 @@ public interface ShuffleWriteHandler {
* Write the blocks to storage
*
* @param shuffleBlocks blocks to storage
* @throws IOException
* @throws IllegalStateException
* @throws Exception
*/
void write(List<ShufflePartitionedBlock> shuffleBlocks) throws IOException, IllegalStateException;
void write(List<ShufflePartitionedBlock> shuffleBlocks) throws Exception;
}
Original file line number Diff line number Diff line change
Expand Up @@ -102,7 +102,7 @@ private void initialize() throws Exception {

@Override
public void write(
List<ShufflePartitionedBlock> shuffleBlocks) throws IOException, IllegalStateException {
List<ShufflePartitionedBlock> shuffleBlocks) throws Exception {
final long start = System.currentTimeMillis();
writeLock.lock();
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ private String pickBasePath(

@Override
public synchronized void write(
List<ShufflePartitionedBlock> shuffleBlocks) throws IOException, IllegalStateException {
List<ShufflePartitionedBlock> shuffleBlocks) throws Exception {

// Ignore this write, if the shuffle directory is deleted after being uploaded in multi mode
// or after its app heartbeat times out.
Expand Down
Loading