-
Notifications
You must be signed in to change notification settings - Fork 170
[#706] Implement spill method to avoid memory deadlock #714
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
Merged
Merged
Changes from all commits
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
137 changes: 137 additions & 0 deletions
137
client-spark/common/src/main/java/org/apache/spark/shuffle/writer/DataPusher.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,137 @@ | ||
| /* | ||
| * 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.writer; | ||
|
|
||
| import java.io.Closeable; | ||
| import java.io.IOException; | ||
| import java.util.Collections; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Optional; | ||
| import java.util.Set; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.ThreadPoolExecutor; | ||
| import java.util.concurrent.TimeUnit; | ||
|
|
||
| import com.google.common.collect.Queues; | ||
| import com.google.common.collect.Sets; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import org.apache.uniffle.client.api.ShuffleWriteClient; | ||
| import org.apache.uniffle.client.response.SendShuffleDataResult; | ||
| import org.apache.uniffle.common.ShuffleBlockInfo; | ||
| import org.apache.uniffle.common.exception.RssException; | ||
| import org.apache.uniffle.common.util.ThreadUtils; | ||
|
|
||
| /** | ||
| * A {@link DataPusher} that is responsible for sending data to remote | ||
| * shuffle servers asynchronously. | ||
| */ | ||
| public class DataPusher implements Closeable { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could DataPusher be used for MapReduce?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I haven't seen this part code.
zuston marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| private static final Logger LOGGER = LoggerFactory.getLogger(DataPusher.class); | ||
|
|
||
| private final ExecutorService executorService; | ||
|
|
||
| private final ShuffleWriteClient shuffleWriteClient; | ||
| // Must be thread safe | ||
| private final Map<String, Set<Long>> taskToSuccessBlockIds; | ||
advancedxy marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| // Must be thread safe | ||
| private final Map<String, Set<Long>> taskToFailedBlockIds; | ||
| private String rssAppId; | ||
| // Must be thread safe | ||
| private final Set<String> failedTaskIds; | ||
|
|
||
| public DataPusher(ShuffleWriteClient shuffleWriteClient, | ||
| Map<String, Set<Long>> taskToSuccessBlockIds, | ||
| Map<String, Set<Long>> taskToFailedBlockIds, | ||
| Set<String> failedTaskIds, | ||
| int threadPoolSize, | ||
| int threadKeepAliveTime) { | ||
| this.shuffleWriteClient = shuffleWriteClient; | ||
| this.taskToSuccessBlockIds = taskToSuccessBlockIds; | ||
| this.taskToFailedBlockIds = taskToFailedBlockIds; | ||
| this.failedTaskIds = failedTaskIds; | ||
| this.executorService = new ThreadPoolExecutor( | ||
| threadPoolSize, | ||
| threadPoolSize * 2, | ||
| threadKeepAliveTime, | ||
| TimeUnit.SECONDS, | ||
| Queues.newLinkedBlockingQueue(Integer.MAX_VALUE), | ||
| ThreadUtils.getThreadFactory(this.getClass().getName()) | ||
| ); | ||
| } | ||
|
|
||
| public CompletableFuture<Long> send(AddBlockEvent event) { | ||
| if (rssAppId == null) { | ||
| throw new RssException("RssAppId should be set."); | ||
| } | ||
| return CompletableFuture.supplyAsync(() -> { | ||
| String taskId = event.getTaskId(); | ||
| List<ShuffleBlockInfo> shuffleBlockInfoList = event.getShuffleDataInfoList(); | ||
| try { | ||
| SendShuffleDataResult result = shuffleWriteClient.sendShuffleData( | ||
| rssAppId, | ||
| shuffleBlockInfoList, | ||
| () -> !isValidTask(taskId) | ||
| ); | ||
| putBlockId(taskToSuccessBlockIds, taskId, result.getSuccessBlockIds()); | ||
| putBlockId(taskToFailedBlockIds, taskId, result.getFailedBlockIds()); | ||
| } finally { | ||
| List<Runnable> callbackChain = Optional.of(event.getProcessedCallbackChain()).orElse(Collections.EMPTY_LIST); | ||
| for (Runnable runnable : callbackChain) { | ||
| runnable.run(); | ||
advancedxy marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
| } | ||
| return shuffleBlockInfoList.stream() | ||
| .map(x -> x.getFreeMemory()) | ||
| .reduce((a, b) -> a + b) | ||
| .get(); | ||
| }, executorService); | ||
| } | ||
|
|
||
| private synchronized void putBlockId( | ||
| Map<String, Set<Long>> taskToBlockIds, | ||
| String taskAttemptId, | ||
| Set<Long> blockIds) { | ||
| if (blockIds == null || blockIds.isEmpty()) { | ||
| return; | ||
| } | ||
| taskToBlockIds.computeIfAbsent(taskAttemptId, x -> Sets.newConcurrentHashSet()).addAll(blockIds); | ||
| } | ||
|
|
||
| public boolean isValidTask(String taskId) { | ||
| return !failedTaskIds.contains(taskId); | ||
| } | ||
|
|
||
| public void setRssAppId(String rssAppId) { | ||
| this.rssAppId = rssAppId; | ||
| } | ||
|
|
||
| @Override | ||
| public void close() throws IOException { | ||
| if (executorService != null) { | ||
| try { | ||
| ThreadUtils.shutdownThreadPool(executorService, 5); | ||
advancedxy marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } catch (InterruptedException interruptedException) { | ||
| LOGGER.error("Errors on shutdown thread pool of [{}].", this.getClass().getSimpleName()); | ||
| } | ||
| } | ||
| } | ||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.