-
Notifications
You must be signed in to change notification settings - Fork 5.5k
Combine spill strategies #16069
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
rschlussel
merged 4 commits into
prestodb:master
from
rschlussel:combine-spill-strategies
May 13, 2021
Merged
Combine spill strategies #16069
Changes from all commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
0b5f138
Remove periodic check from MemoryRevokingScheduler
rschlussel 04c9b8b
Combine memory pool and query based spill strategies
rschlussel 7fe0491
Disable flaky test TestPrestoDriver.testQueryCancelByInterrupt
rschlussel 06aebfd
Disable flaky test in TestJdbcClient
rschlussel 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
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 |
|---|---|---|
|
|
@@ -17,6 +17,7 @@ | |
| import com.facebook.presto.memory.LocalMemoryManager; | ||
| import com.facebook.presto.memory.MemoryPool; | ||
| import com.facebook.presto.memory.MemoryPoolListener; | ||
| import com.facebook.presto.memory.QueryContext; | ||
| import com.facebook.presto.memory.VoidTraversingQueryContextVisitor; | ||
| import com.facebook.presto.operator.OperatorContext; | ||
| import com.facebook.presto.operator.PipelineContext; | ||
|
|
@@ -29,85 +30,93 @@ | |
| import com.google.common.collect.ImmutableList; | ||
| import com.google.common.collect.Ordering; | ||
|
|
||
| import javax.annotation.Nullable; | ||
| import javax.annotation.PostConstruct; | ||
| import javax.annotation.PreDestroy; | ||
| import javax.inject.Inject; | ||
|
|
||
| import java.util.ArrayList; | ||
| import java.util.Collection; | ||
| import java.util.Comparator; | ||
| import java.util.HashMap; | ||
| import java.util.List; | ||
| import java.util.Optional; | ||
| import java.util.concurrent.ScheduledExecutorService; | ||
| import java.util.concurrent.ScheduledFuture; | ||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
| import java.util.SortedMap; | ||
| import java.util.TreeMap; | ||
| import java.util.concurrent.Callable; | ||
| import java.util.concurrent.ExecutorService; | ||
| import java.util.concurrent.atomic.AtomicLong; | ||
| import java.util.function.Function; | ||
| import java.util.function.Supplier; | ||
|
|
||
| import static com.facebook.airlift.concurrent.Threads.threadsNamed; | ||
| import static com.facebook.presto.execution.MemoryRevokingUtils.getMemoryPools; | ||
| import static com.facebook.presto.sql.analyzer.FeaturesConfig.TaskSpillingStrategy.PER_TASK_MEMORY_THRESHOLD; | ||
| import static com.google.common.base.Preconditions.checkArgument; | ||
| import static com.google.common.base.Verify.verify; | ||
| import static com.google.common.collect.ImmutableList.toImmutableList; | ||
| import static java.util.Collections.singletonList; | ||
| import static java.util.Objects.requireNonNull; | ||
| import static java.util.concurrent.TimeUnit.SECONDS; | ||
| import static java.util.concurrent.Executors.newSingleThreadExecutor; | ||
|
|
||
| public class MemoryRevokingScheduler | ||
| { | ||
| private static final Logger log = Logger.get(MemoryRevokingScheduler.class); | ||
|
|
||
| private static final Ordering<SqlTask> ORDER_BY_CREATE_TIME = Ordering.natural().onResultOf(SqlTask::getTaskCreatedTime); | ||
|
|
||
| private final List<MemoryPool> memoryPools; | ||
| private final Function<QueryId, QueryContext> queryContextSupplier; | ||
| private final Supplier<List<SqlTask>> currentTasksSupplier; | ||
| private final ScheduledExecutorService taskManagementExecutor; | ||
| private final ExecutorService memoryRevocationExecutor; | ||
| private final double memoryRevokingThreshold; | ||
| private final double memoryRevokingTarget; | ||
| private final TaskSpillingStrategy spillingStrategy; | ||
|
|
||
| private final List<MemoryPool> memoryPools; | ||
| private final MemoryPoolListener memoryPoolListener = this::onMemoryReserved; | ||
|
|
||
| @Nullable | ||
| private ScheduledFuture<?> scheduledFuture; | ||
|
|
||
| private final AtomicBoolean checkPending = new AtomicBoolean(); | ||
| private final boolean queryLimitSpillEnabled; | ||
|
|
||
| @Inject | ||
| public MemoryRevokingScheduler( | ||
| LocalMemoryManager localMemoryManager, | ||
| SqlTaskManager sqlTaskManager, | ||
| TaskManagementExecutor taskManagementExecutor, | ||
| FeaturesConfig config) | ||
| { | ||
| this( | ||
| ImmutableList.copyOf(getMemoryPools(localMemoryManager)), | ||
| requireNonNull(sqlTaskManager, "sqlTaskManager cannot be null")::getAllTasks, | ||
| requireNonNull(taskManagementExecutor, "taskManagementExecutor cannot be null").getExecutor(), | ||
| requireNonNull(sqlTaskManager, "sqlTaskManager cannot be null")::getQueryContext, | ||
| config.getMemoryRevokingThreshold(), | ||
| config.getMemoryRevokingTarget(), | ||
| config.getTaskSpillingStrategy()); | ||
| config.getTaskSpillingStrategy(), | ||
| config.isQueryLimitSpillEnabled()); | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| MemoryRevokingScheduler( | ||
| List<MemoryPool> memoryPools, | ||
| Supplier<List<SqlTask>> currentTasksSupplier, | ||
| ScheduledExecutorService taskManagementExecutor, | ||
| Function<QueryId, QueryContext> queryContextSupplier, | ||
| double memoryRevokingThreshold, | ||
| double memoryRevokingTarget, | ||
| TaskSpillingStrategy taskSpillingStrategy) | ||
| TaskSpillingStrategy taskSpillingStrategy, | ||
| boolean queryLimitSpillEnabled) | ||
| { | ||
| this.memoryPools = ImmutableList.copyOf(requireNonNull(memoryPools, "memoryPools is null")); | ||
| this.currentTasksSupplier = requireNonNull(currentTasksSupplier, "currentTasksSupplier is null"); | ||
| this.taskManagementExecutor = requireNonNull(taskManagementExecutor, "taskManagementExecutor is null"); | ||
| this.currentTasksSupplier = requireNonNull(currentTasksSupplier, "allTasksSupplier is null"); | ||
| this.queryContextSupplier = requireNonNull(queryContextSupplier, "queryContextSupplier is null"); | ||
| this.memoryRevokingThreshold = checkFraction(memoryRevokingThreshold, "memoryRevokingThreshold"); | ||
| this.memoryRevokingTarget = checkFraction(memoryRevokingTarget, "memoryRevokingTarget"); | ||
| // by using a single thread executor, we don't need to worry about locking to ensure only | ||
| // one revocation request per-query/memory pool is processed at a time. | ||
| this.memoryRevocationExecutor = newSingleThreadExecutor(threadsNamed("memory-revocation")); | ||
| this.spillingStrategy = requireNonNull(taskSpillingStrategy, "taskSpillingStrategy is null"); | ||
| checkArgument(spillingStrategy != PER_TASK_MEMORY_THRESHOLD, "spilling strategy cannot be PER_TASK_MEMORY_THRESHOLD in MemoryRevokingScheduler"); | ||
| checkArgument( | ||
| memoryRevokingTarget <= memoryRevokingThreshold, | ||
| "memoryRevokingTarget should be less than or equal memoryRevokingThreshold, but got %s and %s respectively", | ||
| memoryRevokingTarget, memoryRevokingThreshold); | ||
| this.queryLimitSpillEnabled = queryLimitSpillEnabled; | ||
| } | ||
|
|
||
| private static double checkFraction(double value, String valueName) | ||
|
|
@@ -120,95 +129,131 @@ private static double checkFraction(double value, String valueName) | |
| @PostConstruct | ||
| public void start() | ||
| { | ||
| registerPeriodicCheck(); | ||
| registerPoolListeners(); | ||
| } | ||
|
|
||
| private void registerPeriodicCheck() | ||
| { | ||
| this.scheduledFuture = taskManagementExecutor.scheduleWithFixedDelay(() -> { | ||
| try { | ||
| requestMemoryRevokingIfNeeded(); | ||
| } | ||
| catch (Exception e) { | ||
| log.error(e, "Error requesting system memory revoking"); | ||
| } | ||
| }, 1, 1, SECONDS); | ||
| } | ||
|
|
||
| @PreDestroy | ||
| public void stop() | ||
| { | ||
| if (scheduledFuture != null) { | ||
| scheduledFuture.cancel(true); | ||
| scheduledFuture = null; | ||
| } | ||
|
|
||
| memoryPools.forEach(memoryPool -> memoryPool.removeListener(memoryPoolListener)); | ||
| memoryRevocationExecutor.shutdown(); | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| void registerPoolListeners() | ||
| private void registerPoolListeners() | ||
| { | ||
| memoryPools.forEach(memoryPool -> memoryPool.addListener(memoryPoolListener)); | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| void awaitAsynchronousCallbacksRun() | ||
| throws InterruptedException | ||
| { | ||
| memoryRevocationExecutor.invokeAll(singletonList((Callable<?>) () -> null)); | ||
|
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. Clever, I didn't realize that |
||
| } | ||
|
|
||
| private void onMemoryReserved(MemoryPool memoryPool, QueryId queryId, long queryMemoryReservation) | ||
| { | ||
| try { | ||
| if (!memoryRevokingNeeded(memoryPool)) { | ||
| return; | ||
| if (queryLimitSpillEnabled) { | ||
| QueryContext queryContext = queryContextSupplier.apply(queryId); | ||
| verify(queryContext != null, "QueryContext not found for queryId %s", queryId); | ||
| long maxTotalMemory = queryContext.getMaxTotalMemory(); | ||
| if (memoryRevokingNeededForQuery(queryMemoryReservation, maxTotalMemory)) { | ||
| log.debug("Scheduling check for %s", queryId); | ||
| scheduleQueryRevoking(queryContext, maxTotalMemory); | ||
| } | ||
| } | ||
|
|
||
| if (checkPending.compareAndSet(false, true)) { | ||
| if (memoryRevokingNeededForPool(memoryPool)) { | ||
| log.debug("Scheduling check for %s", memoryPool); | ||
| scheduleRevoking(); | ||
| scheduleMemoryPoolRevoking(memoryPool); | ||
| } | ||
| } | ||
| catch (Exception e) { | ||
| log.error(e, "Error when acting on memory pool reservation"); | ||
| } | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| void requestMemoryRevokingIfNeeded() | ||
| private boolean memoryRevokingNeededForQuery(long queryMemoryReservation, long maxTotalMemory) | ||
| { | ||
| if (checkPending.compareAndSet(false, true)) { | ||
| runMemoryRevoking(); | ||
| } | ||
| return queryMemoryReservation >= maxTotalMemory; | ||
| } | ||
|
|
||
| private void scheduleRevoking() | ||
| private void scheduleQueryRevoking(QueryContext queryContext, long maxTotalMemory) | ||
| { | ||
| taskManagementExecutor.execute(() -> { | ||
| memoryRevocationExecutor.execute(() -> { | ||
| try { | ||
| runMemoryRevoking(); | ||
| revokeQueryMemory(queryContext, maxTotalMemory); | ||
| } | ||
| catch (Exception e) { | ||
| log.error(e, "Error requesting memory revoking"); | ||
| } | ||
| }); | ||
| } | ||
|
|
||
| private synchronized void runMemoryRevoking() | ||
| private void revokeQueryMemory(QueryContext queryContext, long maxTotalMemory) | ||
| { | ||
| if (checkPending.getAndSet(false)) { | ||
| Collection<SqlTask> allTasks = null; | ||
| for (MemoryPool memoryPool : memoryPools) { | ||
| if (!memoryRevokingNeeded(memoryPool)) { | ||
| continue; | ||
| QueryId queryId = queryContext.getQueryId(); | ||
| MemoryPool memoryPool = queryContext.getMemoryPool(); | ||
| // get a fresh value for queryTotalMemory in case it's changed (e.g. by a previous revocation request) | ||
| long queryTotalMemory = getTotalQueryMemoryReservation(queryId, memoryPool); | ||
| // order tasks by decreasing revocableMemory so that we don't spill more tasks than needed | ||
| SortedMap<Long, TaskContext> queryTaskContextsMap = new TreeMap<>(Comparator.reverseOrder()); | ||
| queryContext.getAllTaskContexts() | ||
| .forEach(taskContext -> queryTaskContextsMap.put(taskContext.getTaskMemoryContext().getRevocableMemory(), taskContext)); | ||
|
|
||
| AtomicLong remainingBytesToRevoke = new AtomicLong(queryTotalMemory - maxTotalMemory); | ||
| Collection<TaskContext> queryTaskContexts = queryTaskContextsMap.values(); | ||
| remainingBytesToRevoke.addAndGet(-MemoryRevokingSchedulerUtils.getMemoryAlreadyBeingRevoked(queryTaskContexts, remainingBytesToRevoke.get())); | ||
| for (TaskContext taskContext : queryTaskContexts) { | ||
| if (remainingBytesToRevoke.get() <= 0) { | ||
| break; | ||
| } | ||
| taskContext.accept(new VoidTraversingQueryContextVisitor<AtomicLong>() | ||
| { | ||
| @Override | ||
| public Void visitOperatorContext(OperatorContext operatorContext, AtomicLong remainingBytesToRevoke) | ||
| { | ||
| if (remainingBytesToRevoke.get() > 0) { | ||
| long revokedBytes = operatorContext.requestMemoryRevoking(); | ||
| if (revokedBytes > 0) { | ||
| remainingBytesToRevoke.addAndGet(-revokedBytes); | ||
| log.debug("taskId=%s: requested revoking %s; remaining %s", taskContext.getTaskId(), revokedBytes, remainingBytesToRevoke); | ||
| } | ||
| } | ||
| return null; | ||
| } | ||
| }, remainingBytesToRevoke); | ||
| } | ||
| } | ||
|
|
||
| if (allTasks == null) { | ||
| allTasks = requireNonNull(currentTasksSupplier.get()); | ||
| } | ||
| private static long getTotalQueryMemoryReservation(QueryId queryId, MemoryPool memoryPool) | ||
| { | ||
| return memoryPool.getQueryMemoryReservation(queryId) + memoryPool.getQueryRevocableMemoryReservation(queryId); | ||
| } | ||
|
|
||
| requestMemoryRevoking(memoryPool, allTasks); | ||
| private void scheduleMemoryPoolRevoking(MemoryPool memoryPool) | ||
| { | ||
| memoryRevocationExecutor.execute(() -> { | ||
| try { | ||
| runMemoryPoolRevoking(memoryPool); | ||
| } | ||
| catch (Exception e) { | ||
| log.error(e, "Error requesting memory revoking"); | ||
| } | ||
| }); | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| void runMemoryPoolRevoking(MemoryPool memoryPool) | ||
| { | ||
| if (!memoryRevokingNeededForPool(memoryPool)) { | ||
| return; | ||
| } | ||
| Collection<SqlTask> allTasks = requireNonNull(currentTasksSupplier.get()); | ||
| requestMemoryPoolRevoking(memoryPool, allTasks); | ||
| } | ||
|
|
||
| private void requestMemoryRevoking(MemoryPool memoryPool, Collection<SqlTask> allTasks) | ||
| private void requestMemoryPoolRevoking(MemoryPool memoryPool, Collection<SqlTask> allTasks) | ||
| { | ||
| long remainingBytesToRevoke = (long) (-memoryPool.getFreeBytes() + (memoryPool.getMaxBytes() * (1.0 - memoryRevokingTarget))); | ||
| ArrayList<SqlTask> runningTasksInPool = findRunningTasksInMemoryPool(allTasks, memoryPool); | ||
|
|
@@ -218,7 +263,7 @@ private void requestMemoryRevoking(MemoryPool memoryPool, Collection<SqlTask> al | |
| } | ||
| } | ||
|
|
||
| private boolean memoryRevokingNeeded(MemoryPool memoryPool) | ||
| private boolean memoryRevokingNeededForPool(MemoryPool memoryPool) | ||
| { | ||
| return memoryPool.getReservedRevocableBytes() > 0 | ||
| && memoryPool.getFreeBytes() <= memoryPool.getMaxBytes() * (1.0 - memoryRevokingThreshold); | ||
|
|
||
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.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Now that this instance is created in the constructor, it should be shutdown in the
stop()method to avoid leaking it.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It wasn't created in the constructor for the tests, but I'll actually just change the tests to get the executor from here, and that will remove any need to do any checking about the pool size.