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 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
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,12 @@ 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-one-partition")
zuston marked this conversation as resolved.
Show resolved Hide resolved
.intType()
.defaultValue(1)
.withDescription("The max concurrency of single one partition, the data partition size is equals to this value.");

public ShuffleServerConf() {
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,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,15 +99,16 @@ public void updateWriteMetrics(StorageWriteMetrics metrics) {
ShuffleWriteHandler newWriteHandler(CreateShuffleWriteHandlerRequest request) {
try {
String user = request.getUser();
return new HdfsShuffleWriteHandler(
return new PooledHdfsShuffleWriteHandler(
zuston marked this conversation as resolved.
Show resolved Hide resolved
request.getAppId(),
request.getShuffleId(),
request.getStartPartition(),
request.getEndPartition(),
storagePath,
request.getFileNamePrefix(),
conf,
user
user,
request.getMaxFileNumber()
);
} catch (Exception e) {
throw new RuntimeException(e);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
/*
* 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.storage.handler.impl;

import java.io.IOException;
import java.util.List;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;

import org.apache.hadoop.conf.Configuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.uniffle.common.ShufflePartitionedBlock;
import org.apache.uniffle.storage.handler.api.ShuffleWriteHandler;
import org.apache.uniffle.storage.util.ShuffleStorageUtils;

public class PooledHdfsShuffleWriteHandler implements ShuffleWriteHandler {
private static final Logger LOGGER = LoggerFactory.getLogger(PooledHdfsShuffleWriteHandler.class);

private final BlockingQueue<HdfsShuffleWriteHandler> queue;
private final int maxConcurrency;
private final String basePath;

public PooledHdfsShuffleWriteHandler(
String appId,
int shuffleId,
int startPartition,
int endPartition,
String storageBasePath,
String fileNamePrefix,
Configuration hadoopConf,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe we can get the parameter concurrency from hadooConf. The hadoopConf we can pass from our client. It will be more flexible.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes. Although it’s a little bit strange.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have added the todo comment to support this in the future.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you create some issues for these todo?

Copy link
Member Author

@zuston zuston Dec 12, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes. I will do after this PR is merged.

String user,
int concurrency) {
this.maxConcurrency = concurrency;
this.queue = new LinkedBlockingQueue<>(maxConcurrency);
this.basePath = ShuffleStorageUtils.getFullShuffleDataFolder(storageBasePath,
ShuffleStorageUtils.getShuffleDataPath(appId, shuffleId, startPartition, endPartition));

// todo: support init lazily
try {
for (int i = 0; i < maxConcurrency; i++) {
queue.add(
new HdfsShuffleWriteHandler(
appId,
shuffleId,
startPartition,
endPartition,
storageBasePath,
fileNamePrefix + "_" + i,
hadoopConf,
user
)
);
}
} catch (Exception e) {
throw new RuntimeException("Errors on initializing Hdfs writer handler.", e);
}
}

@Override
public void write(List<ShufflePartitionedBlock> shuffleBlocks) throws IOException, IllegalStateException {
if (queue.isEmpty()) {
LOGGER.warn("No free hdfs writer handler, it will wait. storage path: {}", basePath);
}
HdfsShuffleWriteHandler writeHandler = queue.poll();
writeHandler.write(shuffleBlocks);
queue.add(writeHandler);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ public class CreateShuffleWriteHandlerRequest {
private Configuration conf;
private int storageDataReplica;
private String user;
private int maxFileNumber;

public CreateShuffleWriteHandlerRequest(
String storageType,
Expand All @@ -43,6 +44,33 @@ public CreateShuffleWriteHandlerRequest(
Configuration conf,
int storageDataReplica,
String user) {
this(
storageType,
appId,
shuffleId,
startPartition,
endPartition,
storageBasePaths,
fileNamePrefix,
conf,
storageDataReplica,
user,
1
);
}

public CreateShuffleWriteHandlerRequest(
String storageType,
String appId,
int shuffleId,
int startPartition,
int endPartition,
String[] storageBasePaths,
String fileNamePrefix,
Configuration conf,
int storageDataReplica,
String user,
int maxFileNumber) {
this.storageType = storageType;
this.appId = appId;
this.shuffleId = shuffleId;
Expand All @@ -53,6 +81,7 @@ public CreateShuffleWriteHandlerRequest(
this.conf = conf;
this.storageDataReplica = storageDataReplica;
this.user = user;
this.maxFileNumber = maxFileNumber;
}

public String getStorageType() {
Expand Down Expand Up @@ -98,4 +127,8 @@ public String getUser() {
public void setUser(String user) {
this.user = user;
}

public int getMaxFileNumber() {
return maxFileNumber;
}
}