
io.prestosql.operator.Driver Maven / Gradle / Ivy
/*
* Licensed 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 io.prestosql.operator;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.VerifyException;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
import io.airlift.log.Logger;
import io.airlift.units.Duration;
import io.prestosql.execution.ScheduledSplit;
import io.prestosql.execution.TaskSource;
import io.prestosql.metadata.Split;
import io.prestosql.spi.Page;
import io.prestosql.spi.PrestoException;
import io.prestosql.spi.connector.UpdatablePageSource;
import io.prestosql.sql.planner.plan.PlanNodeId;
import javax.annotation.concurrent.GuardedBy;
import java.io.Closeable;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Supplier;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.base.Throwables.throwIfUnchecked;
import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
import static io.airlift.concurrent.MoreFutures.getFutureValue;
import static io.prestosql.operator.Operator.NOT_BLOCKED;
import static io.prestosql.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR;
import static java.lang.Boolean.TRUE;
import static java.util.Objects.requireNonNull;
//
// NOTE: As a general strategy the methods should "stage" a change and only
// process the actual change before lock release (DriverLockResult.close()).
// The assures that only one thread will be working with the operators at a
// time and state changer threads are not blocked.
//
public class Driver
implements Closeable
{
private static final Logger log = Logger.get(Driver.class);
private final DriverContext driverContext;
private final List activeOperators;
// this is present only for debugging
@SuppressWarnings("unused")
private final List allOperators;
private final Optional sourceOperator;
private final Optional deleteOperator;
// This variable acts as a staging area. When new splits (encapsulated in TaskSource) are
// provided to a Driver, the Driver will not process them right away. Instead, the splits are
// added to this staging area. This staging area will be drained asynchronously. That's when
// the new splits get processed.
private final AtomicReference pendingTaskSourceUpdates = new AtomicReference<>();
private final Map> revokingOperators = new HashMap<>();
private final AtomicReference state = new AtomicReference<>(State.ALIVE);
private final DriverLock exclusiveLock = new DriverLock();
@GuardedBy("exclusiveLock")
private TaskSource currentTaskSource;
private final AtomicReference> driverBlockedFuture = new AtomicReference<>();
private enum State
{
ALIVE, NEED_DESTRUCTION, DESTROYED
}
public static Driver createDriver(DriverContext driverContext, List operators)
{
requireNonNull(driverContext, "driverContext is null");
requireNonNull(operators, "operators is null");
Driver driver = new Driver(driverContext, operators);
driver.initialize();
return driver;
}
@VisibleForTesting
public static Driver createDriver(DriverContext driverContext, Operator firstOperator, Operator... otherOperators)
{
requireNonNull(driverContext, "driverContext is null");
requireNonNull(firstOperator, "firstOperator is null");
requireNonNull(otherOperators, "otherOperators is null");
ImmutableList operators = ImmutableList.builder()
.add(firstOperator)
.add(otherOperators)
.build();
return createDriver(driverContext, operators);
}
private Driver(DriverContext driverContext, List operators)
{
this.driverContext = requireNonNull(driverContext, "driverContext is null");
this.allOperators = ImmutableList.copyOf(requireNonNull(operators, "operators is null"));
checkArgument(allOperators.size() > 1, "At least two operators are required");
this.activeOperators = new ArrayList<>(operators);
checkArgument(!operators.isEmpty(), "There must be at least one operator");
Optional sourceOperator = Optional.empty();
Optional deleteOperator = Optional.empty();
for (Operator operator : operators) {
if (operator instanceof SourceOperator) {
checkArgument(sourceOperator.isEmpty(), "There must be at most one SourceOperator");
sourceOperator = Optional.of((SourceOperator) operator);
}
else if (operator instanceof DeleteOperator) {
checkArgument(deleteOperator.isEmpty(), "There must be at most one DeleteOperator");
deleteOperator = Optional.of((DeleteOperator) operator);
}
}
this.sourceOperator = sourceOperator;
this.deleteOperator = deleteOperator;
currentTaskSource = sourceOperator.map(operator -> new TaskSource(operator.getSourceId(), ImmutableSet.of(), false)).orElse(null);
// initially the driverBlockedFuture is not blocked (it is completed)
SettableFuture> future = SettableFuture.create();
future.set(null);
driverBlockedFuture.set(future);
}
// the memory revocation request listeners are added here in a separate initialize() method
// instead of the constructor to prevent leaking the "this" reference to
// another thread, which will cause unsafe publication of this instance.
private void initialize()
{
activeOperators.stream()
.map(Operator::getOperatorContext)
.forEach(operatorContext -> operatorContext.setMemoryRevocationRequestListener(() -> driverBlockedFuture.get().set(null)));
}
public DriverContext getDriverContext()
{
return driverContext;
}
public Optional getSourceId()
{
return sourceOperator.map(SourceOperator::getSourceId);
}
@Override
public void close()
{
// mark the service for destruction
if (!state.compareAndSet(State.ALIVE, State.NEED_DESTRUCTION)) {
return;
}
exclusiveLock.interruptCurrentOwner();
// if we can get the lock, attempt a clean shutdown; otherwise someone else will shutdown
tryWithLock(() -> TRUE);
}
public boolean isFinished()
{
checkLockNotHeld("Cannot check finished status while holding the driver lock");
// if we can get the lock, attempt a clean shutdown; otherwise someone else will shutdown
Optional result = tryWithLock(this::isFinishedInternal);
return result.orElseGet(() -> state.get() != State.ALIVE || driverContext.isDone());
}
@GuardedBy("exclusiveLock")
private boolean isFinishedInternal()
{
checkLockHeld("Lock must be held to call isFinishedInternal");
boolean finished = state.get() != State.ALIVE || driverContext.isDone() || activeOperators.isEmpty() || activeOperators.get(activeOperators.size() - 1).isFinished();
if (finished) {
state.compareAndSet(State.ALIVE, State.NEED_DESTRUCTION);
}
return finished;
}
public void updateSource(TaskSource sourceUpdate)
{
checkLockNotHeld("Cannot update sources while holding the driver lock");
checkArgument(
sourceOperator.isPresent() && sourceOperator.get().getSourceId().equals(sourceUpdate.getPlanNodeId()),
"sourceUpdate is for a plan node that is different from this Driver's source node");
// stage the new updates
pendingTaskSourceUpdates.updateAndGet(current -> current == null ? sourceUpdate : current.update(sourceUpdate));
// attempt to get the lock and process the updates we staged above
// updates will be processed in close if and only if we got the lock
tryWithLock(() -> TRUE);
}
@GuardedBy("exclusiveLock")
private void processNewSources()
{
checkLockHeld("Lock must be held to call processNewSources");
// only update if the driver is still alive
if (state.get() != State.ALIVE) {
return;
}
TaskSource sourceUpdate = pendingTaskSourceUpdates.getAndSet(null);
if (sourceUpdate == null) {
return;
}
// merge the current source and the specified source update
TaskSource newSource = currentTaskSource.update(sourceUpdate);
// if the update contains no new data, just return
if (newSource == currentTaskSource) {
return;
}
// determine new splits to add
Set newSplits = Sets.difference(newSource.getSplits(), currentTaskSource.getSplits());
// add new splits
SourceOperator sourceOperator = this.sourceOperator.orElseThrow(VerifyException::new);
for (ScheduledSplit newSplit : newSplits) {
Split split = newSplit.getSplit();
Supplier> pageSource = sourceOperator.addSplit(split);
deleteOperator.ifPresent(deleteOperator -> deleteOperator.setPageSource(pageSource));
}
// set no more splits
if (newSource.isNoMoreSplits()) {
sourceOperator.noMoreSplits();
}
currentTaskSource = newSource;
}
public ListenableFuture> processFor(Duration duration)
{
checkLockNotHeld("Cannot process for a duration while holding the driver lock");
requireNonNull(duration, "duration is null");
// if the driver is blocked we don't need to continue
SettableFuture> blockedFuture = driverBlockedFuture.get();
if (!blockedFuture.isDone()) {
return blockedFuture;
}
long maxRuntime = duration.roundTo(TimeUnit.NANOSECONDS);
Optional> result = tryWithLock(100, TimeUnit.MILLISECONDS, () -> {
OperationTimer operationTimer = createTimer();
driverContext.startProcessTimer();
driverContext.getYieldSignal().setWithDelay(maxRuntime, driverContext.getYieldExecutor());
try {
long start = System.nanoTime();
do {
ListenableFuture> future = processInternal(operationTimer);
if (!future.isDone()) {
return updateDriverBlockedFuture(future);
}
}
while (System.nanoTime() - start < maxRuntime && !isFinishedInternal());
}
finally {
driverContext.getYieldSignal().reset();
driverContext.recordProcessed(operationTimer);
}
return NOT_BLOCKED;
});
return result.orElse(NOT_BLOCKED);
}
public ListenableFuture> process()
{
checkLockNotHeld("Cannot process while holding the driver lock");
// if the driver is blocked we don't need to continue
SettableFuture> blockedFuture = driverBlockedFuture.get();
if (!blockedFuture.isDone()) {
return blockedFuture;
}
Optional> result = tryWithLock(100, TimeUnit.MILLISECONDS, () -> {
ListenableFuture> future = processInternal(createTimer());
return updateDriverBlockedFuture(future);
});
return result.orElse(NOT_BLOCKED);
}
private OperationTimer createTimer()
{
return new OperationTimer(
driverContext.isCpuTimerEnabled(),
driverContext.isCpuTimerEnabled() && driverContext.isPerOperatorCpuTimerEnabled());
}
private ListenableFuture> updateDriverBlockedFuture(ListenableFuture> sourceBlockedFuture)
{
// driverBlockedFuture will be completed as soon as the sourceBlockedFuture is completed
// or any of the operators gets a memory revocation request
SettableFuture> newDriverBlockedFuture = SettableFuture.create();
driverBlockedFuture.set(newDriverBlockedFuture);
sourceBlockedFuture.addListener(() -> newDriverBlockedFuture.set(null), directExecutor());
// it's possible that memory revoking is requested for some operator
// before we update driverBlockedFuture above and we don't want to miss that
// notification, so we check to see whether that's the case before returning.
boolean memoryRevokingRequested = activeOperators.stream()
.filter(operator -> !revokingOperators.containsKey(operator))
.map(Operator::getOperatorContext)
.anyMatch(OperatorContext::isMemoryRevokingRequested);
if (memoryRevokingRequested) {
newDriverBlockedFuture.set(null);
}
return newDriverBlockedFuture;
}
@GuardedBy("exclusiveLock")
private ListenableFuture> processInternal(OperationTimer operationTimer)
{
checkLockHeld("Lock must be held to call processInternal");
handleMemoryRevoke();
try {
processNewSources();
// If there is only one operator, finish it
// Some operators (LookupJoinOperator and HashBuildOperator) are broken and requires finish to be called continuously
// TODO remove the second part of the if statement, when these operators are fixed
// Note: finish should not be called on the natural source of the pipeline as this could cause the task to finish early
if (!activeOperators.isEmpty() && activeOperators.size() != allOperators.size()) {
Operator rootOperator = activeOperators.get(0);
rootOperator.finish();
rootOperator.getOperatorContext().recordFinish(operationTimer);
}
boolean movedPage = false;
for (int i = 0; i < activeOperators.size() - 1 && !driverContext.isDone(); i++) {
Operator current = activeOperators.get(i);
Operator next = activeOperators.get(i + 1);
// skip blocked operator
if (getBlockedFuture(current).isPresent()) {
continue;
}
// if the current operator is not finished and next operator isn't blocked and needs input...
if (!current.isFinished() && getBlockedFuture(next).isEmpty() && next.needsInput()) {
// get an output page from current operator
Page page = current.getOutput();
current.getOperatorContext().recordGetOutput(operationTimer, page);
// if we got an output page, add it to the next operator
if (page != null && page.getPositionCount() != 0) {
next.addInput(page);
next.getOperatorContext().recordAddInput(operationTimer, page);
movedPage = true;
}
if (current instanceof SourceOperator) {
movedPage = true;
}
}
// if current operator is finished...
if (current.isFinished()) {
// let next operator know there will be no more data
next.finish();
next.getOperatorContext().recordFinish(operationTimer);
}
}
for (int index = activeOperators.size() - 1; index >= 0; index--) {
if (activeOperators.get(index).isFinished()) {
// close and remove this operator and all source operators
List finishedOperators = this.activeOperators.subList(0, index + 1);
Throwable throwable = closeAndDestroyOperators(finishedOperators);
finishedOperators.clear();
if (throwable != null) {
throwIfUnchecked(throwable);
throw new RuntimeException(throwable);
}
// Finish the next operator, which is now the first operator.
if (!activeOperators.isEmpty()) {
Operator newRootOperator = activeOperators.get(0);
newRootOperator.finish();
newRootOperator.getOperatorContext().recordFinish(operationTimer);
}
break;
}
}
// if we did not move any pages, check if we are blocked
if (!movedPage) {
List blockedOperators = new ArrayList<>();
List> blockedFutures = new ArrayList<>();
for (Operator operator : activeOperators) {
Optional> blocked = getBlockedFuture(operator);
if (blocked.isPresent()) {
blockedOperators.add(operator);
blockedFutures.add(blocked.get());
}
}
if (!blockedFutures.isEmpty()) {
// unblock when the first future is complete
ListenableFuture> blocked = firstFinishedFuture(blockedFutures);
// driver records serial blocked time
driverContext.recordBlocked(blocked);
// each blocked operator is responsible for blocking the execution
// until one of the operators can continue
for (Operator operator : blockedOperators) {
operator.getOperatorContext().recordBlocked(blocked);
}
return blocked;
}
}
return NOT_BLOCKED;
}
catch (Throwable t) {
List interrupterStack = exclusiveLock.getInterrupterStack();
if (interrupterStack == null) {
driverContext.failed(t);
throw t;
}
// Driver thread was interrupted which should only happen if the task is already finished.
// If this becomes the actual cause of a failed query there is a bug in the task state machine.
Exception exception = new Exception("Interrupted By");
exception.setStackTrace(interrupterStack.stream().toArray(StackTraceElement[]::new));
PrestoException newException = new PrestoException(GENERIC_INTERNAL_ERROR, "Driver was interrupted", exception);
newException.addSuppressed(t);
driverContext.failed(newException);
throw newException;
}
}
@GuardedBy("exclusiveLock")
private void handleMemoryRevoke()
{
for (int i = 0; i < activeOperators.size() && !driverContext.isDone(); i++) {
Operator operator = activeOperators.get(i);
if (revokingOperators.containsKey(operator)) {
checkOperatorFinishedRevoking(operator);
}
else if (operator.getOperatorContext().isMemoryRevokingRequested()) {
ListenableFuture> future = operator.startMemoryRevoke();
revokingOperators.put(operator, future);
checkOperatorFinishedRevoking(operator);
}
}
}
@GuardedBy("exclusiveLock")
private void checkOperatorFinishedRevoking(Operator operator)
{
ListenableFuture> future = revokingOperators.get(operator);
if (future.isDone()) {
getFutureValue(future); // propagate exception if there was some
revokingOperators.remove(operator);
operator.finishMemoryRevoke();
operator.getOperatorContext().resetMemoryRevokingRequested();
}
}
@GuardedBy("exclusiveLock")
private void destroyIfNecessary()
{
checkLockHeld("Lock must be held to call destroyIfNecessary");
if (!state.compareAndSet(State.NEED_DESTRUCTION, State.DESTROYED)) {
return;
}
// if we get an error while closing a driver, record it and we will throw it at the end
Throwable inFlightException = null;
try {
inFlightException = closeAndDestroyOperators(activeOperators);
if (driverContext.getMemoryUsage() > 0) {
log.error("Driver still has memory reserved after freeing all operator memory.");
}
if (driverContext.getSystemMemoryUsage() > 0) {
log.error("Driver still has system memory reserved after freeing all operator memory.");
}
if (driverContext.getRevocableMemoryUsage() > 0) {
log.error("Driver still has revocable memory reserved after freeing all operator memory. Freeing it.");
}
driverContext.finished();
}
catch (Throwable t) {
// this shouldn't happen but be safe
inFlightException = addSuppressedException(
inFlightException,
t,
"Error destroying driver for task %s",
driverContext.getTaskId());
}
if (inFlightException != null) {
// this will always be an Error or Runtime
throwIfUnchecked(inFlightException);
throw new RuntimeException(inFlightException);
}
}
private Throwable closeAndDestroyOperators(List operators)
{
// record the current interrupted status (and clear the flag); we'll reset it later
boolean wasInterrupted = Thread.interrupted();
Throwable inFlightException = null;
try {
for (Operator operator : operators) {
try {
operator.close();
}
catch (InterruptedException t) {
// don't record the stack
wasInterrupted = true;
}
catch (Throwable t) {
inFlightException = addSuppressedException(
inFlightException,
t,
"Error closing operator %s for task %s",
operator.getOperatorContext().getOperatorId(),
driverContext.getTaskId());
}
try {
operator.getOperatorContext().destroy();
}
catch (Throwable t) {
inFlightException = addSuppressedException(
inFlightException,
t,
"Error freeing all allocated memory for operator %s for task %s",
operator.getOperatorContext().getOperatorId(),
driverContext.getTaskId());
}
}
}
finally {
// reset the interrupted flag
if (wasInterrupted) {
Thread.currentThread().interrupt();
}
}
return inFlightException;
}
private Optional> getBlockedFuture(Operator operator)
{
ListenableFuture> blocked = revokingOperators.get(operator);
if (blocked != null) {
// We mark operator as blocked regardless of blocked.isDone(), because finishMemoryRevoke has not been called yet.
return Optional.of(blocked);
}
blocked = operator.isBlocked();
if (!blocked.isDone()) {
return Optional.of(blocked);
}
blocked = operator.getOperatorContext().isWaitingForMemory();
if (!blocked.isDone()) {
return Optional.of(blocked);
}
blocked = operator.getOperatorContext().isWaitingForRevocableMemory();
if (!blocked.isDone()) {
return Optional.of(blocked);
}
return Optional.empty();
}
private static Throwable addSuppressedException(Throwable inFlightException, Throwable newException, String message, Object... args)
{
if (newException instanceof Error) {
if (inFlightException == null) {
inFlightException = newException;
}
else {
// Self-suppression not permitted
if (inFlightException != newException) {
inFlightException.addSuppressed(newException);
}
}
}
else {
// log normal exceptions instead of rethrowing them
log.error(newException, message, args);
}
return inFlightException;
}
private synchronized void checkLockNotHeld(String message)
{
checkState(!exclusiveLock.isHeldByCurrentThread(), message);
}
@GuardedBy("exclusiveLock")
private synchronized void checkLockHeld(String message)
{
checkState(exclusiveLock.isHeldByCurrentThread(), message);
}
private static ListenableFuture> firstFinishedFuture(List> futures)
{
if (futures.size() == 1) {
return futures.get(0);
}
SettableFuture> result = SettableFuture.create();
for (ListenableFuture> future : futures) {
future.addListener(() -> result.set(null), directExecutor());
}
return result;
}
// Note: task cannot return null
private Optional tryWithLock(Supplier task)
{
return tryWithLock(0, TimeUnit.MILLISECONDS, task);
}
// Note: task cannot return null
private Optional tryWithLock(long timeout, TimeUnit unit, Supplier task)
{
checkLockNotHeld("Lock cannot be reacquired");
boolean acquired = false;
try {
acquired = exclusiveLock.tryLock(timeout, unit);
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
if (!acquired) {
return Optional.empty();
}
Optional result;
try {
result = Optional.of(task.get());
}
finally {
try {
try {
processNewSources();
}
finally {
destroyIfNecessary();
}
}
finally {
exclusiveLock.unlock();
}
}
// If there are more source updates available, attempt to reacquire the lock and process them.
// This can happen if new sources are added while we're holding the lock here doing work.
// NOTE: this is separate duplicate code to make debugging lock reacquisition easier
// The first condition is for processing the pending updates if this driver is still ALIVE
// The second condition is to destroy the driver if the state is NEED_DESTRUCTION
while (((pendingTaskSourceUpdates.get() != null && state.get() == State.ALIVE) || state.get() == State.NEED_DESTRUCTION)
&& exclusiveLock.tryLock()) {
try {
try {
processNewSources();
}
finally {
destroyIfNecessary();
}
}
finally {
exclusiveLock.unlock();
}
}
return result;
}
private static class DriverLock
{
private final ReentrantLock lock = new ReentrantLock();
@GuardedBy("this")
private Thread currentOwner;
@GuardedBy("this")
private List interrupterStack;
public boolean isHeldByCurrentThread()
{
return lock.isHeldByCurrentThread();
}
public boolean tryLock()
{
checkState(!lock.isHeldByCurrentThread(), "Lock is not reentrant");
boolean acquired = lock.tryLock();
if (acquired) {
setOwner();
}
return acquired;
}
public boolean tryLock(long timeout, TimeUnit unit)
throws InterruptedException
{
checkState(!lock.isHeldByCurrentThread(), "Lock is not reentrant");
boolean acquired = lock.tryLock(timeout, unit);
if (acquired) {
setOwner();
}
return acquired;
}
private synchronized void setOwner()
{
checkState(lock.isHeldByCurrentThread(), "Current thread does not hold lock");
currentOwner = Thread.currentThread();
// NOTE: We do not use interrupted stack information to know that another
// thread has attempted to interrupt the driver, and interrupt this new lock
// owner. The interrupted stack information is for debugging purposes only.
// In the case of interruption, the caller should (and does) have a separate
// state to prevent further processing in the Driver.
}
public synchronized void unlock()
{
checkState(lock.isHeldByCurrentThread(), "Current thread does not hold lock");
currentOwner = null;
lock.unlock();
}
public synchronized List getInterrupterStack()
{
return interrupterStack;
}
public synchronized void interruptCurrentOwner()
{
// there is a benign race condition here were the lock holder
// can be change between attempting to get lock and grabbing
// the synchronized lock here, but in either case we want to
// interrupt the lock holder thread
if (interrupterStack == null) {
interrupterStack = ImmutableList.copyOf(Thread.currentThread().getStackTrace());
}
if (currentOwner != null) {
currentOwner.interrupt();
}
}
}
}
© 2015 - 2025 Weber Informatics LLC | Privacy Policy