-
Notifications
You must be signed in to change notification settings - Fork 5.5k
Interrupt runaway splits #16111
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
Interrupt runaway splits #16111
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,6 +23,7 @@ | |
| import com.facebook.presto.execution.TaskId; | ||
| import com.facebook.presto.execution.TaskManagerConfig; | ||
| import com.facebook.presto.execution.TaskManagerConfig.TaskPriorityTracking; | ||
| import com.facebook.presto.operator.scalar.JoniRegexpFunctions; | ||
| import com.facebook.presto.server.ServerConfig; | ||
| import com.facebook.presto.spi.PrestoException; | ||
| import com.facebook.presto.spi.QueryId; | ||
|
|
@@ -60,11 +61,11 @@ | |
| import java.util.concurrent.RejectedExecutionException; | ||
| import java.util.concurrent.ScheduledExecutorService; | ||
| import java.util.concurrent.ThreadPoolExecutor; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.atomic.AtomicLong; | ||
| import java.util.concurrent.atomic.AtomicLongArray; | ||
| import java.util.function.DoubleSupplier; | ||
| import java.util.function.Function; | ||
| import java.util.function.Predicate; | ||
|
|
||
| import static com.facebook.airlift.concurrent.Threads.daemonThreadsNamed; | ||
| import static com.facebook.airlift.concurrent.Threads.threadsNamed; | ||
|
|
@@ -74,19 +75,28 @@ | |
| import static com.google.common.base.Preconditions.checkArgument; | ||
| import static com.google.common.base.Preconditions.checkState; | ||
| import static com.google.common.collect.Sets.newConcurrentHashSet; | ||
| import static java.lang.System.lineSeparator; | ||
| import static java.util.Arrays.asList; | ||
| import static java.util.Objects.requireNonNull; | ||
| import static java.util.concurrent.Executors.newCachedThreadPool; | ||
| import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor; | ||
| import static java.util.concurrent.TimeUnit.MICROSECONDS; | ||
| import static java.util.concurrent.TimeUnit.NANOSECONDS; | ||
| import static java.util.concurrent.TimeUnit.SECONDS; | ||
| import static java.util.stream.Collectors.joining; | ||
|
|
||
| @ThreadSafe | ||
| public class TaskExecutor | ||
| { | ||
| private static final Logger log = Logger.get(TaskExecutor.class); | ||
|
|
||
| // print out split call stack if it has been running for a certain amount of time | ||
| private static final Duration LONG_SPLIT_WARNING_THRESHOLD = new Duration(600, TimeUnit.SECONDS); | ||
| private static final Duration LONG_SPLIT_WARNING_THRESHOLD = new Duration(600, SECONDS); | ||
| // Interrupt a split if it is running longer than this AND it's blocked on something known | ||
| private static final Predicate<List<StackTraceElement>> DEFAULT_INTERRUPTIBLE_SPLIT_PREDICATE = elements -> | ||
|
||
| elements.stream() | ||
| .anyMatch(element -> element.getClassName().equals(JoniRegexpFunctions.class.getName())); | ||
| private static final Duration DEFAULT_INTERRUPT_SPLIT_INTERVAL = new Duration(60, SECONDS); | ||
|
|
||
| private static final AtomicLong NEXT_RUNNER_ID = new AtomicLong(); | ||
|
|
||
|
|
@@ -101,6 +111,10 @@ public class TaskExecutor | |
|
|
||
| private final Ticker ticker; | ||
|
|
||
| private final Duration interruptRunawaySplitsTimeout; | ||
| private final Predicate<List<StackTraceElement>> interruptibleSplitPredicate; | ||
| private final Duration interruptSplitInterval; | ||
|
|
||
| private final ScheduledExecutorService splitMonitorExecutor = newSingleThreadScheduledExecutor(daemonThreadsNamed("TaskExecutor")); | ||
| private final SortedSet<RunningSplitInfo> runningSplitInfos = new ConcurrentSkipListSet<>(); | ||
|
|
||
|
|
@@ -174,6 +188,9 @@ public TaskExecutor(TaskManagerConfig config, EmbedVersion embedVersion, Multile | |
| config.getMinDriversPerTask(), | ||
| config.getMaxDriversPerTask(), | ||
| config.getTaskPriorityTracking(), | ||
| config.getInterruptRunawaySplitsTimeout(), | ||
| DEFAULT_INTERRUPTIBLE_SPLIT_PREDICATE, | ||
| DEFAULT_INTERRUPT_SPLIT_INTERVAL, | ||
| embedVersion, | ||
| splitQueue, | ||
| Ticker.systemTicker()); | ||
|
|
@@ -194,6 +211,9 @@ public TaskExecutor( | |
| guaranteedNumberOfDriversPerTask, | ||
| maximumNumberOfDriversPerTask, | ||
| taskPriorityTracking, | ||
| new TaskManagerConfig().getInterruptRunawaySplitsTimeout(), | ||
| DEFAULT_INTERRUPTIBLE_SPLIT_PREDICATE, | ||
| DEFAULT_INTERRUPT_SPLIT_INTERVAL, | ||
|
||
| new EmbedVersion(new ServerConfig()), | ||
| new MultilevelSplitQueue(2), ticker); | ||
| } | ||
|
|
@@ -214,6 +234,9 @@ public TaskExecutor( | |
| guaranteedNumberOfDriversPerTask, | ||
| maximumNumberOfDriversPerTask, | ||
| taskPriorityTracking, | ||
| new TaskManagerConfig().getInterruptRunawaySplitsTimeout(), | ||
| DEFAULT_INTERRUPTIBLE_SPLIT_PREDICATE, | ||
| DEFAULT_INTERRUPT_SPLIT_INTERVAL, | ||
| new EmbedVersion(new ServerConfig()), | ||
| splitQueue, | ||
| ticker); | ||
|
|
@@ -226,6 +249,9 @@ public TaskExecutor( | |
| int guaranteedNumberOfDriversPerTask, | ||
| int maximumNumberOfDriversPerTask, | ||
| TaskPriorityTracking taskPriorityTracking, | ||
| Duration interruptRunawaySplitsTimeout, | ||
| Predicate<List<StackTraceElement>> interruptibleSplitPredicate, | ||
| Duration interruptSplitInterval, | ||
| EmbedVersion embedVersion, | ||
| MultilevelSplitQueue splitQueue, | ||
| Ticker ticker) | ||
|
|
@@ -234,6 +260,8 @@ public TaskExecutor( | |
| checkArgument(guaranteedNumberOfDriversPerTask > 0, "guaranteedNumberOfDriversPerTask must be at least 1"); | ||
| checkArgument(maximumNumberOfDriversPerTask > 0, "maximumNumberOfDriversPerTask must be at least 1"); | ||
| checkArgument(guaranteedNumberOfDriversPerTask <= maximumNumberOfDriversPerTask, "guaranteedNumberOfDriversPerTask cannot be greater than maximumNumberOfDriversPerTask"); | ||
| checkArgument(interruptRunawaySplitsTimeout.getValue(SECONDS) >= 1.0, "interruptRunawaySplitsTimeout must be at least 1 second"); | ||
| checkArgument(interruptSplitInterval.getValue(SECONDS) >= 1.0, "interruptSplitInterval must be at least 1 second"); | ||
|
|
||
| // we manage thread pool size directly, so create an unlimited pool | ||
| this.executor = newCachedThreadPool(threadsNamed("task-processor-%s")); | ||
|
|
@@ -263,6 +291,9 @@ public TaskExecutor( | |
| } | ||
| this.taskPriorityTrackerFactory = taskPriorityTrackerFactory; | ||
| this.tasks = new LinkedList<>(); | ||
| this.interruptRunawaySplitsTimeout = interruptRunawaySplitsTimeout; | ||
| this.interruptibleSplitPredicate = interruptibleSplitPredicate; | ||
| this.interruptSplitInterval = interruptSplitInterval; | ||
| } | ||
|
|
||
| @PostConstruct | ||
|
|
@@ -272,6 +303,10 @@ public synchronized void start() | |
| for (int i = 0; i < runnerThreads; i++) { | ||
| addRunnerThread(); | ||
| } | ||
| if (interruptRunawaySplitsTimeout != null) { | ||
| long interval = (long) interruptSplitInterval.getValue(SECONDS); | ||
| splitMonitorExecutor.scheduleAtFixedRate(this::interruptRunawaySplits, interval, interval, SECONDS); | ||
| } | ||
| } | ||
|
|
||
| @PreDestroy | ||
|
|
@@ -514,6 +549,25 @@ private synchronized PrioritizedSplitRunner pollNextSplitWorker() | |
| return null; | ||
| } | ||
|
|
||
| private void interruptRunawaySplits() | ||
| { | ||
| for (RunningSplitInfo splitInfo : runningSplitInfos) { | ||
| Duration duration = Duration.succinctNanos(ticker.read() - splitInfo.getStartTime()); | ||
| if (duration.compareTo(interruptRunawaySplitsTimeout) < 0) { | ||
| continue; | ||
| } | ||
|
|
||
| List<StackTraceElement> stack = asList(splitInfo.getThread().getStackTrace()); | ||
| if (interruptibleSplitPredicate.test(stack)) { | ||
| String stackString = stack.stream() | ||
| .map(Object::toString) | ||
| .collect(joining(lineSeparator())); | ||
| log.warn("Interrupting runaway split " + splitInfo.getSplitInfo() + lineSeparator() + stackString); | ||
| splitInfo.getThread().interrupt(); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private class TaskRunner | ||
| implements Runnable | ||
| { | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.